diff --git a/dev-tools/eclipse/dot.classpath b/dev-tools/eclipse/dot.classpath
index da680b88bfb..056e5aa5607 100644
--- a/dev-tools/eclipse/dot.classpath
+++ b/dev-tools/eclipse/dot.classpath
@@ -100,7 +100,7 @@
-
+
@@ -115,7 +115,7 @@
-
+
diff --git a/dev-tools/maven/pom.xml.template b/dev-tools/maven/pom.xml.template
index 5c62f5f2289..e783c02ad1b 100644
--- a/dev-tools/maven/pom.xml.template
+++ b/dev-tools/maven/pom.xml.template
@@ -283,7 +283,7 @@
org.apache.zookeeper
zookeeper
- 3.3.3
+ 3.3.4
org.carrot2
@@ -670,7 +670,7 @@
solr-noggit
${project.version}
jar
- solr/lib/apache-solr-noggit-r1209632.jar
+ solr/lib/apache-solr-noggit-r1211150.jar
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0ef90c7a5ed..221061fa115 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -28,7 +28,7 @@ Apache Tika 1.0
Carrot2 3.5.0
Velocity 1.6.4 and Velocity Tools 2.0
Apache UIMA 2.3.1
-Apache ZooKeeper 3.3.3
+Apache ZooKeeper 3.3.4
Upgrading from Solr 3.6-dev
diff --git a/solr/build.xml b/solr/build.xml
index 8b76d1cfc6c..7e1a3c4657d 100644
--- a/solr/build.xml
+++ b/solr/build.xml
@@ -482,7 +482,7 @@
-
+
diff --git a/solr/cloud-dev/example1.sh b/solr/cloud-dev/example1.sh
new file mode 100644
index 00000000000..dfd73008f4e
--- /dev/null
+++ b/solr/cloud-dev/example1.sh
@@ -0,0 +1,25 @@
+#!/usr/bin/env bash
+
+cd ..
+
+rm -r -f example2
+
+rm -r -f dist
+rm -r -f build
+rm -r -f example/solr/zoo_data
+rm -f example/example.log
+
+ant example dist
+
+cp -r -f example example2
+
+
+cd example
+java -DzkRun -DnumShards=2 -DSTOP.PORT=7983 -DSTOP.KEY=key -Dbootstrap_confdir=solr/conf -jar start.jar 1>example.log 2>&1 &
+
+sleep 10
+
+cd ../example2
+java -Djetty.port=9574 -DzkRun -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6574 -DSTOP.KEY=key -jar start.jar 1>example2.log 2>&1 &
+
+
diff --git a/solr/cloud-dev/example2.sh b/solr/cloud-dev/example2.sh
new file mode 100644
index 00000000000..582a3fd0485
--- /dev/null
+++ b/solr/cloud-dev/example2.sh
@@ -0,0 +1,34 @@
+#!/usr/bin/env bash
+
+cd ..
+
+rm -r -f example2
+rm -r -f example3
+rm -r -f example4
+
+rm -r -f dist
+rm -r -f build
+rm -r -f example/solr/zoo_data
+rm -f example/example.log
+
+ant example dist
+
+cp -r -f example example2
+cp -r -f example example3
+cp -r -f example example4
+
+
+cd example
+java -DzkRun -DnumShards=2 -DSTOP.PORT=7983 -DSTOP.KEY=key -Dbootstrap_confdir=solr/conf -jar start.jar 1>example.log 2>&1 &
+
+sleep 10
+
+cd ../example2
+java -Djetty.port=9574 -DzkRun -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6574 -DSTOP.KEY=key -jar start.jar 1>example2.log 2>&1 &
+
+cd ../example3
+java -Djetty.port=9575 -DzkRun -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6575 -DSTOP.KEY=key -jar start.jar 1>example3.log 2>&1 &
+
+cd ../example4
+java -Djetty.port=9576 -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6576 -DSTOP.KEY=key -jar start.jar 1>example4.log 2>&1 &
+
diff --git a/solr/cloud-dev/example3.sh b/solr/cloud-dev/example3.sh
new file mode 100644
index 00000000000..0d5ede7e059
--- /dev/null
+++ b/solr/cloud-dev/example3.sh
@@ -0,0 +1,33 @@
+#!/usr/bin/env bash
+
+cd ..
+
+rm -r -f example2
+rm -r -f example3
+rm -r -f example4
+
+rm -r -f dist
+rm -r -f build
+rm -r -f example/solr/zoo_data
+rm -f example/example.log
+
+ant example dist
+
+cp -r -f example example2
+cp -r -f example example3
+cp -r -f example example4
+
+
+cd example
+java -DzkRun -DnumShards=2 -DSTOP.PORT=7983 -DSTOP.KEY=key -Dbootstrap_confdir=solr/conf -DzkHost=localhost:9983,localhost:14574,localhost:14585 -jar start.jar 1>example.log 2>&1 &
+
+sleep 10
+
+cd ../example2
+java -Djetty.port=13574 -DzkRun -DzkHost=localhost:9983,localhost:14574,localhost:14575 -DnumShards=2 -DSTOP.PORT=6574 -DSTOP.KEY=key -jar start.jar 1>example2.log 2>&1 &
+
+cd ../example3
+java -Djetty.port=13585 -DzkRun -DzkHost=localhost:9983,localhost:14574,localhost:14585 -DnumShards=2 -DSTOP.PORT=6575 -DSTOP.KEY=key -jar start.jar 1>example3.log 2>&1 &
+
+cd ../example4
+java -Djetty.port=13596 -DzkHost=localhost:9983,localhost:14574,localhost:14585 -DnumShards=2 -DSTOP.PORT=6576 -DSTOP.KEY=key -jar start.jar 1>example4.log 2>&1 &
diff --git a/solr/cloud-dev/solrcloud.sh b/solr/cloud-dev/solrcloud.sh
new file mode 100644
index 00000000000..3dc88ff606c
--- /dev/null
+++ b/solr/cloud-dev/solrcloud.sh
@@ -0,0 +1,42 @@
+#!/usr/bin/env bash
+
+cd ..
+
+rm -r -f example2
+rm -r -f example3
+rm -r -f example4
+rm -r -f example5
+rm -r -f example6
+
+rm -r -f dist
+rm -r -f build
+rm -r -f example/solr/zoo_data
+rm -f example/example.log
+
+ant example dist
+
+cp -r -f example example2
+cp -r -f example example3
+cp -r -f example example4
+cp -r -f example example5
+cp -r -f example example6
+
+java -classpath lib/*:dist/*:build/lucene-libs/* org.apache.solr.cloud.ZkController 127.0.0.1:9983 example/solr 8983 example/solr/conf conf1
+
+cd example
+java -DzkRun -DnumShards=2 -DSTOP.PORT=7983 -DSTOP.KEY=key -jar start.jar 1>example.log 2>&1 &
+
+cd ../example2
+java -Djetty.port=7574 -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6574 -DSTOP.KEY=key -jar start.jar 1>example2.log 2>&1 &
+
+cd ../example3
+java -Djetty.port=7575 -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6575 -DSTOP.KEY=key -jar start.jar 1>example3.log 2>&1 &
+
+cd ../example4
+java -Djetty.port=7576 -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6576 -DSTOP.KEY=key -jar start.jar 1>example4.log 2>&1 &
+
+cd ../example5
+java -Djetty.port=7577 -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6577 -DSTOP.KEY=key -jar start.jar 1>example5.log 2>&1 &
+
+cd ../example6
+java -Djetty.port=7578 -DzkHost=localhost:9983 -DnumShards=2 -DSTOP.PORT=6578 -DSTOP.KEY=key -jar start.jar 1>example6.log 2>&1 &
diff --git a/solr/cloud-dev/stop.sh b/solr/cloud-dev/stop.sh
new file mode 100644
index 00000000000..0e22e64a24d
--- /dev/null
+++ b/solr/cloud-dev/stop.sh
@@ -0,0 +1,10 @@
+#!/usr/bin/env bash
+
+cd ../example
+
+java -DSTOP.PORT=7983 -DSTOP.KEY=key -jar start.jar --stop
+java -DSTOP.PORT=6574 -DSTOP.KEY=key -jar start.jar --stop
+java -DSTOP.PORT=6575 -DSTOP.KEY=key -jar start.jar --stop
+java -DSTOP.PORT=6576 -DSTOP.KEY=key -jar start.jar --stop
+java -DSTOP.PORT=6577 -DSTOP.KEY=key -jar start.jar --stop
+java -DSTOP.PORT=6578 -DSTOP.KEY=key -jar start.jar --stop
\ No newline at end of file
diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java
index 15686ace24f..01a360bd571 100644
--- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java
+++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java
@@ -81,7 +81,7 @@ public class SolrWriter extends DIHWriterBase implements DIHWriter {
try {
log.info("Deleting document: " + id);
DeleteUpdateCommand delCmd = new DeleteUpdateCommand(req);
- delCmd.id = id.toString();
+ delCmd.setId(id.toString());
processor.processDelete(delCmd);
} catch (IOException e) {
log.error("Exception while deleteing: " + id, e);
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java
index 027a9c3a49b..3098875f351 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java
@@ -173,9 +173,8 @@ public class TestContentStreamDataSource extends AbstractDataImportHandlerTestCa
}
private JettySolrRunner createJetty(SolrInstance instance) throws Exception {
- System.setProperty("solr.solr.home", instance.getHomeDir());
System.setProperty("solr.data.dir", instance.getDataDir());
- JettySolrRunner jetty = new JettySolrRunner("/solr", 0);
+ JettySolrRunner jetty = new JettySolrRunner(instance.getHomeDir(), "/solr", 0);
jetty.start();
return jetty;
}
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
index e1c8a27c15d..dbd7df085f7 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
@@ -47,7 +47,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
private static Logger LOG = LoggerFactory.getLogger(TestSolrEntityProcessorEndToEnd.class);
- private static final String SOLR_SOURCE_URL = "http://localhost:8983/solr";
+ //rivate static final String SOLR_SOURCE_URL = "http://localhost:8983/solr";
private static final String SOLR_CONFIG = "dataimport-solrconfig.xml";
private static final String SOLR_SCHEMA = "dataimport-schema.xml";
private static final String SOLR_HOME = "dih/solr";
@@ -68,29 +68,36 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
solrDoc.put("desc", "SolrDescription");
SOLR_DOCS.add(solrDoc);
}
-
- private static final String DIH_CONFIG_TAGS_INNER_ENTITY = "\r\n"
- + " \r\n"
- + " \r\n"
- + " \r\n"
- + " \r\n"
- + " \r\n"
- + " \r\n"
- + " \r\n"
- + " \r\n" + " \r\n"
- + " \r\n" + " \r\n" + "\r\n";
+
private SolrInstance instance = null;
private JettySolrRunner jetty;
- private static String generateDIHConfig(String options) {
+ private static String getDihConfigTagsInnerEntity(int port) {
+ return "\r\n"
+ + " \r\n"
+ + " \r\n"
+ + " \r\n"
+ + " \r\n"
+ + " \r\n"
+ + " \r\n"
+ + " \r\n"
+ + " \r\n" + " \r\n"
+ + " \r\n" + " \r\n" + "\r\n";
+ }
+
+ private static String generateDIHConfig(String options, int port) {
return "\r\n" + " \r\n"
+ " \r\n" + " \r\n"
+ + getSourceUrl(port) + "' " + options + " />\r\n" + " \r\n"
+ "\r\n";
}
+ private static String getSourceUrl(int port) {
+ return "http://localhost:" + port + "/solr";
+ }
+
//TODO: fix this test to close its directories
static String savedFactory;
@BeforeClass
@@ -107,7 +114,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
System.setProperty("solr.directoryFactory", savedFactory);
}
}
-
+
@Override
@Before
public void setUp() throws Exception {
@@ -138,7 +145,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
try {
addDocumentsToSolr(SOLR_DOCS);
- runFullImport(generateDIHConfig("query='*:*' rows='2' fields='id,desc' onError='skip'"));
+ runFullImport(generateDIHConfig("query='*:*' rows='2' fields='id,desc' onError='skip'", jetty.getLocalPort()));
} catch (Exception e) {
LOG.error(e.getMessage(), e);
fail(e.getMessage());
@@ -156,7 +163,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
addDocumentsToSolr(generateSolrDocuments(30));
Map map = new HashMap();
map.put("rows", "50");
- runFullImport(generateDIHConfig("query='*:*' fq='desc:Description1*,desc:Description*2' rows='2'"), map);
+ runFullImport(generateDIHConfig("query='*:*' fq='desc:Description1*,desc:Description*2' rows='2'", jetty.getLocalPort()), map);
} catch (Exception e) {
LOG.error(e.getMessage(), e);
fail(e.getMessage());
@@ -171,7 +178,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
try {
addDocumentsToSolr(generateSolrDocuments(7));
- runFullImport(generateDIHConfig("query='*:*' fields='id' rows='2'"));
+ runFullImport(generateDIHConfig("query='*:*' fields='id' rows='2'", jetty.getLocalPort()));
} catch (Exception e) {
LOG.error(e.getMessage(), e);
fail(e.getMessage());
@@ -197,7 +204,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
try {
MockDataSource.setIterator("select * from x", DB_DOCS.iterator());
addDocumentsToSolr(SOLR_DOCS);
- runFullImport(DIH_CONFIG_TAGS_INNER_ENTITY);
+ runFullImport(getDihConfigTagsInnerEntity(jetty.getLocalPort()));
} catch (Exception e) {
LOG.error(e.getMessage(), e);
fail(e.getMessage());
@@ -224,7 +231,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
assertQ(req("*:*"), "//result[@numFound='0']");
try {
- runFullImport(generateDIHConfig("query='*:*' rows='2' fields='id,desc' onError='skip'"));
+ runFullImport(generateDIHConfig("query='*:*' rows='2' fields='id,desc' onError='skip'", jetty.getLocalPort()));
} catch (Exception e) {
LOG.error(e.getMessage(), e);
fail(e.getMessage());
@@ -237,7 +244,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
assertQ(req("*:*"), "//result[@numFound='0']");
try {
- runFullImport(generateDIHConfig("query='bogus:3' rows='2' fields='id,desc' onError='abort'"));
+ runFullImport(generateDIHConfig("query='bogus:3' rows='2' fields='id,desc' onError='abort'", jetty.getLocalPort()));
} catch (Exception e) {
LOG.error(e.getMessage(), e);
fail(e.getMessage());
@@ -255,8 +262,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
addDocumentsToSolr(docList);
Map map = new HashMap();
map.put("rows", "50");
- runFullImport(generateDIHConfig("query='*:*' rows='6' numThreads='4'"),
- map);
+ runFullImport(generateDIHConfig("query='*:*' rows='6' numThreads='4'", jetty.getLocalPort()), map);
} catch (Exception e) {
LOG.error(e.getMessage(), e);
fail(e.getMessage());
@@ -287,7 +293,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
}
HttpClient client = new HttpClient(new MultiThreadedHttpConnectionManager());
- URL url = new URL(SOLR_SOURCE_URL);
+ URL url = new URL(getSourceUrl(jetty.getLocalPort()));
CommonsHttpSolrServer solrServer = new CommonsHttpSolrServer(url, client);
solrServer.add(sidl);
solrServer.commit(true, true);
@@ -343,9 +349,8 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
}
private JettySolrRunner createJetty(SolrInstance instance) throws Exception {
- System.setProperty("solr.solr.home", instance.getHomeDir());
System.setProperty("solr.data.dir", instance.getDataDir());
- JettySolrRunner jetty = new JettySolrRunner("/solr", 8983);
+ JettySolrRunner jetty = new JettySolrRunner(instance.getHomeDir(), "/solr", 0);
jetty.start();
return jetty;
}
diff --git a/solr/core/build.xml b/solr/core/build.xml
index ef38c354f0d..fcd3259f422 100644
--- a/solr/core/build.xml
+++ b/solr/core/build.xml
@@ -27,6 +27,6 @@
jar.file="${common-solr.dir}/lib/commons-csv-1.0-SNAPSHOT-r966014.jar" />
+ jar.file="${common-solr.dir}/lib/apache-solr-noggit-r1211150.jar" />
diff --git a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
index f656c2866c5..c66c58f00fc 100644
--- a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
+++ b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
@@ -34,6 +34,7 @@ import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.FilterHolder;
import org.mortbay.jetty.servlet.HashSessionIdManager;
import org.mortbay.log.Logger;
+import org.mortbay.thread.QueuedThreadPool;
/**
* Run solr using jetty
@@ -48,30 +49,76 @@ public class JettySolrRunner {
String context;
private String solrConfigFilename;
+ private String schemaFilename;
private boolean waitOnSolr = false;
- public JettySolrRunner(String context, int port) {
- this.init(context, port);
+ private int lastPort = -1;
+
+ private String shards;
+
+ private String dataDir;
+
+ private volatile boolean startedBefore = false;
+
+ private String solrHome;
+
+ private boolean stopAtShutdown;
+
+ public JettySolrRunner(String solrHome, String context, int port) {
+ this.init(solrHome, context, port, true);
}
- public JettySolrRunner(String context, int port, String solrConfigFilename) {
- this.init(context, port);
+ public JettySolrRunner(String solrHome, String context, int port, String solrConfigFilename, String schemaFileName) {
+ this.init(solrHome, context, port, true);
this.solrConfigFilename = solrConfigFilename;
+ this.schemaFilename = schemaFileName;
+ }
+
+ public JettySolrRunner(String solrHome, String context, int port,
+ String solrConfigFilename, String schemaFileName, boolean stopAtShutdown) {
+ this.init(solrHome, context, port, stopAtShutdown);
+ this.solrConfigFilename = solrConfigFilename;
+ this.schemaFilename = schemaFileName;
}
- private void init(String context, int port) {
+ private void init(String solrHome, String context, int port, boolean stopAtShutdown) {
this.context = context;
server = new Server(port);
- server.setStopAtShutdown(true);
+ this.solrHome = solrHome;
+ this.stopAtShutdown = stopAtShutdown;
+ server.setStopAtShutdown(stopAtShutdown);
+ if (!stopAtShutdown) {
+ server.setGracefulShutdown(0);
+ }
+ System.setProperty("solr.solr.home", solrHome);
if (System.getProperty("jetty.testMode") != null) {
// SelectChannelConnector connector = new SelectChannelConnector();
// Normal SocketConnector is what solr's example server uses by default
SocketConnector connector = new SocketConnector();
connector.setPort(port);
connector.setReuseAddress(true);
- server.setConnectors(new Connector[] { connector });
+ if (!stopAtShutdown) {
+ QueuedThreadPool threadPool = (QueuedThreadPool) connector
+ .getThreadPool();
+ if (threadPool != null) {
+ threadPool.setMaxStopTimeMs(100);
+ }
+ }
+ server.setConnectors(new Connector[] {connector});
server.setSessionIdManager(new HashSessionIdManager(new Random()));
+ } else {
+ if (!stopAtShutdown) {
+ for (Connector connector : server.getConnectors()) {
+ if (connector instanceof SocketConnector) {
+ QueuedThreadPool threadPool = (QueuedThreadPool) ((SocketConnector) connector)
+ .getThreadPool();
+ if (threadPool != null) {
+ threadPool.setMaxStopTimeMs(100);
+ }
+ }
+ }
+ }
}
// Initialize the servlets
@@ -92,13 +139,20 @@ public class JettySolrRunner {
}
public void lifeCycleStarted(LifeCycle arg0) {
- System.setProperty("hostPort", Integer.toString(getLocalPort()));
- if (solrConfigFilename != null)
- System.setProperty("solrconfig", solrConfigFilename);
+ lastPort = getFirstConnectorPort();
+ System.setProperty("hostPort", Integer.toString(lastPort));
+ if (solrConfigFilename != null) System.setProperty("solrconfig",
+ solrConfigFilename);
+ if (schemaFilename != null) System.setProperty("schema",
+ schemaFilename);
+// SolrDispatchFilter filter = new SolrDispatchFilter();
+// FilterHolder fh = new FilterHolder(filter);
dispatchFilter = root.addFilter(SolrDispatchFilter.class, "*",
Handler.REQUEST);
- if (solrConfigFilename != null)
- System.clearProperty("solrconfig");
+ if (solrConfigFilename != null) System.clearProperty("solrconfig");
+ if (schemaFilename != null) System.clearProperty("schema");
+ System.clearProperty("solr.solr.home");
+
}
public void lifeCycleFailure(LifeCycle arg0, Throwable arg1) {
@@ -111,6 +165,18 @@ public class JettySolrRunner {
}
+ public FilterHolder getDispatchFilter() {
+ return dispatchFilter;
+ }
+
+ public boolean isRunning() {
+ return server.isRunning();
+ }
+
+ public boolean isStopped() {
+ return server.isStopped();
+ }
+
// ------------------------------------------------------------------------------------------------
// ------------------------------------------------------------------------------------------------
@@ -119,6 +185,21 @@ public class JettySolrRunner {
}
public void start(boolean waitForSolr) throws Exception {
+ // if started before, make a new server
+ if (startedBefore) {
+ waitOnSolr = false;
+ init(solrHome, context, lastPort, stopAtShutdown);
+ } else {
+ startedBefore = true;
+ }
+
+ if( dataDir != null) {
+ System.setProperty("solr.data.dir", dataDir);
+ }
+ if(shards != null) {
+ System.setProperty("shard", shards);
+ }
+
if (!server.isRunning()) {
server.start();
}
@@ -131,27 +212,42 @@ public class JettySolrRunner {
}
}
}
+
+ System.clearProperty("shard");
+ System.clearProperty("solr.data.dir");
}
public void stop() throws Exception {
- if (server.isRunning()) {
+ if (!server.isStopped() && !server.isStopping()) {
server.stop();
- server.join();
}
+ server.join();
}
/**
- * Returns the Local Port of the first Connector found for the jetty Server.
+ * Returns the Local Port of the jetty Server.
*
* @exception RuntimeException if there is no Connector
*/
- public int getLocalPort() {
+ private int getFirstConnectorPort() {
Connector[] conns = server.getConnectors();
if (0 == conns.length) {
throw new RuntimeException("Jetty Server has no Connectors");
}
return conns[0].getLocalPort();
}
+
+ /**
+ * Returns the Local Port of the jetty Server.
+ *
+ * @exception RuntimeException if there is no Connector
+ */
+ public int getLocalPort() {
+ if (lastPort == -1) {
+ throw new IllegalStateException("You cannot get the port until this instance has started");
+ }
+ return lastPort;
+ }
// --------------------------------------------------------------
// --------------------------------------------------------------
@@ -172,12 +268,20 @@ public class JettySolrRunner {
*/
public static void main(String[] args) {
try {
- JettySolrRunner jetty = new JettySolrRunner("/solr", 8983);
+ JettySolrRunner jetty = new JettySolrRunner(".", "/solr", 8983);
jetty.start();
} catch (Exception ex) {
ex.printStackTrace();
}
}
+
+ public void setShards(String shardList) {
+ this.shards = shardList;
+ }
+
+ public void setDataDir(String dataDir) {
+ this.dataDir = dataDir;
+ }
}
class NoLog implements Logger {
diff --git a/solr/core/src/java/org/apache/solr/cloud/AssignShard.java b/solr/core/src/java/org/apache/solr/cloud/AssignShard.java
new file mode 100644
index 00000000000..2343673380f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/AssignShard.java
@@ -0,0 +1,81 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.zookeeper.KeeperException;
+
+public class AssignShard {
+
+ /**
+ * Assign a new unique id up to slices count - then add replicas evenly.
+ *
+ * @param collection
+ *
+ * @param slices
+ * @return
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public static String assignShard(String collection, CloudState state) {
+
+ int shards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP,1);
+
+ String returnShardId = null;
+ Map sliceMap = state.getSlices(collection);
+
+ if (sliceMap == null) {
+ return "shard1";
+ }
+
+ List shardIdNames = new ArrayList(sliceMap.keySet());
+
+ if (shardIdNames.size() < shards) {
+ return "shard" + (shardIdNames.size() + 1);
+ }
+
+ // else figure out which shard needs more replicas
+ final Map map = new HashMap();
+ for (String shardId : shardIdNames) {
+ int cnt = sliceMap.get(shardId).getShards().size();
+ map.put(shardId, cnt);
+ }
+
+ Collections.sort(shardIdNames, new Comparator() {
+
+ @Override
+ public int compare(String o1, String o2) {
+ Integer one = map.get(o1);
+ Integer two = map.get(o2);
+ return one.compareTo(two);
+ }
+ });
+
+ returnShardId = shardIdNames.get(0);
+ return returnShardId;
+ }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
index df583d22c5b..4f083a72f1b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
@@ -1,8 +1,5 @@
package org.apache.solr.cloud;
-import org.apache.solr.common.params.SolrParams;
-
-
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@@ -20,11 +17,14 @@ import org.apache.solr.common.params.SolrParams;
* limitations under the License.
*/
+import org.apache.solr.common.params.SolrParams;
+
public class CloudDescriptor {
private String shardId;
private String collectionName;
private SolrParams params;
-
+ private String roles = "";
+
public void setShardId(String shardId) {
this.shardId = shardId;
}
@@ -41,6 +41,14 @@ public class CloudDescriptor {
this.collectionName = collectionName;
}
+ public String getRoles(){
+ return roles;
+ }
+
+ public void setRoles(String roles){
+ this.roles = roles;
+ }
+
/** Optional parameters that can change how a core is created. */
public SolrParams getParams() {
return params;
diff --git a/solr/core/src/java/org/apache/solr/cloud/CurrentCoreDescriptorProvider.java b/solr/core/src/java/org/apache/solr/cloud/CurrentCoreDescriptorProvider.java
new file mode 100644
index 00000000000..2349df3ada9
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/CurrentCoreDescriptorProvider.java
@@ -0,0 +1,29 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.solr.core.CoreDescriptor;
+
+/**
+ * Provide the current list of registered {@link CoreDescriptor}s.
+ */
+public abstract class CurrentCoreDescriptorProvider {
+ public abstract List getCurrentDescriptors();
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
new file mode 100644
index 00000000000..fe425a5f401
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -0,0 +1,222 @@
+package org.apache.solr.cloud;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
+
+/**
+ * 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.
+ */
+
+public abstract class ElectionContext {
+
+ final String electionPath;
+ final ZkNodeProps leaderProps;
+ final String id;
+ final String leaderPath;
+
+ public ElectionContext(final String shardZkNodeName,
+ final String electionPath, final String leaderPath, final ZkNodeProps leaderProps) {
+ this.id = shardZkNodeName;
+ this.electionPath = electionPath;
+ this.leaderPath = leaderPath;
+ this.leaderProps = leaderProps;
+ }
+
+ abstract void runLeaderProcess(String leaderSeqPath, boolean weAreReplacement) throws KeeperException, InterruptedException, IOException;
+}
+
+class ShardLeaderElectionContextBase extends ElectionContext {
+
+ protected final SolrZkClient zkClient;
+ protected String shardId;
+ protected String collection;
+ protected LeaderElector leaderElector;
+
+ public ShardLeaderElectionContextBase(LeaderElector leaderElector, final String shardId,
+ final String collection, final String shardZkNodeName, ZkNodeProps props, ZkStateReader zkStateReader) {
+ super(shardZkNodeName, ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/leader_elect/"
+ + shardId, ZkStateReader.getShardLeadersPath(collection, shardId),
+ props);
+ this.leaderElector = leaderElector;
+ this.zkClient = zkStateReader.getZkClient();
+ this.shardId = shardId;
+ this.collection = collection;
+ }
+
+ @Override
+ void runLeaderProcess(String leaderSeqPath, boolean weAreReplacement)
+ throws KeeperException, InterruptedException, IOException {
+
+ try {
+ zkClient.makePath(leaderPath,
+ leaderProps == null ? null : ZkStateReader.toJSON(leaderProps),
+ CreateMode.EPHEMERAL, true);
+ } catch (NodeExistsException e) {
+ // if a previous leader ephemeral still exists for some reason, try and
+ // remove it
+ zkClient.delete(leaderPath, -1, true);
+ zkClient.makePath(leaderPath,
+ leaderProps == null ? null : ZkStateReader.toJSON(leaderProps),
+ CreateMode.EPHEMERAL, true);
+ }
+ }
+
+}
+
+// add core container and stop passing core around...
+final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
+ private ZkController zkController;
+ private CoreContainer cc;
+ private SyncStrategy syncStrategy = new SyncStrategy();
+
+ public ShardLeaderElectionContext(LeaderElector leaderElector,
+ final String shardId, final String collection,
+ final String shardZkNodeName, ZkNodeProps props, ZkController zkController, CoreContainer cc) {
+ super(leaderElector, shardId, collection, shardZkNodeName, props,
+ zkController.getZkStateReader());
+ this.zkController = zkController;
+ this.cc = cc;
+ }
+
+ @Override
+ void runLeaderProcess(String leaderSeqPath, boolean weAreReplacement)
+ throws KeeperException, InterruptedException, IOException {
+ if (cc != null) {
+ SolrCore core = null;
+ String coreName = leaderProps.get(ZkStateReader.CORE_NAME_PROP);
+ try {
+ core = cc.getCore(coreName);
+ if (core == null) {
+ throw new SolrException(ErrorCode.SERVER_ERROR, "Core not found:" + coreName);
+ }
+ // should I be leader?
+ if (weAreReplacement && !shouldIBeLeader(leaderProps)) {
+ System.out.println("there is a better leader candidate it appears");
+ rejoinLeaderElection(leaderSeqPath, core);
+ return;
+ }
+
+ if (weAreReplacement) {
+ if (zkClient.exists(leaderPath, true)) {
+ zkClient.delete(leaderPath, -1, true);
+ }
+// System.out.println("I may be the new Leader:" + leaderPath
+// + " - I need to try and sync");
+ boolean success = syncStrategy.sync(zkController, core, leaderProps);
+ if (!success) {
+ // TODO: what if no one can be the leader in a loop?
+ // perhaps we look down the list and if no one is active, we
+ // accept leader role anyhow
+ core.getUpdateHandler().getSolrCoreState().doRecovery(core);
+
+ rejoinLeaderElection(leaderSeqPath, core);
+ return;
+ }
+ }
+
+ // If I am going to be the leader I have to be active
+
+ core.getUpdateHandler().getSolrCoreState().cancelRecovery();
+ zkController.publish(core, ZkStateReader.ACTIVE);
+
+ } finally {
+ if (core != null) {
+ core.close();
+ }
+ }
+
+ }
+
+ super.runLeaderProcess(leaderSeqPath, weAreReplacement);
+ }
+
+ private void rejoinLeaderElection(String leaderSeqPath, SolrCore core)
+ throws InterruptedException, KeeperException, IOException {
+ // remove our ephemeral and re join the election
+ // System.out.println("sync failed, delete our election node:"
+ // + leaderSeqPath);
+ zkController.publish(core, ZkStateReader.DOWN);
+ zkClient.delete(leaderSeqPath, -1, true);
+
+ core.getUpdateHandler().getSolrCoreState().doRecovery(core);
+
+ leaderElector.joinElection(this);
+ }
+
+ private boolean shouldIBeLeader(ZkNodeProps leaderProps) {
+ CloudState cloudState = zkController.getZkStateReader().getCloudState();
+ Map slices = cloudState.getSlices(this.collection);
+ Slice slice = slices.get(shardId);
+ Map shards = slice.getShards();
+ boolean foundSomeoneElseActive = false;
+ for (Map.Entry shard : shards.entrySet()) {
+ String state = shard.getValue().get(ZkStateReader.STATE_PROP);
+
+ if (new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals(
+ new ZkCoreNodeProps(leaderProps).getCoreUrl())) {
+ if (state.equals(ZkStateReader.ACTIVE)
+ && cloudState.liveNodesContain(shard.getValue().get(
+ ZkStateReader.NODE_NAME_PROP))) {
+ // we are alive
+ return true;
+ }
+ }
+
+ if ((state.equals(ZkStateReader.ACTIVE))
+ && cloudState.liveNodesContain(shard.getValue().get(
+ ZkStateReader.NODE_NAME_PROP))
+ && !new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals(
+ new ZkCoreNodeProps(leaderProps).getCoreUrl())) {
+ foundSomeoneElseActive = true;
+ }
+ }
+
+ return !foundSomeoneElseActive;
+ }
+
+}
+
+final class OverseerElectionContext extends ElectionContext {
+
+ private final SolrZkClient zkClient;
+ private final ZkStateReader stateReader;
+
+ public OverseerElectionContext(final String zkNodeName, SolrZkClient zkClient, ZkStateReader stateReader) {
+ super(zkNodeName, "/overseer_elect", null, null);
+ this.zkClient = zkClient;
+ this.stateReader = stateReader;
+ }
+
+ @Override
+ void runLeaderProcess(String leaderSeqPath, boolean weAreReplacement) throws KeeperException, InterruptedException {
+ new Overseer(zkClient, stateReader);
+ }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
new file mode 100644
index 00000000000..48229dd591d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -0,0 +1,286 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.ConnectionLossException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Leader Election process. This class contains the logic by which a
+ * leader is chosen. First call * {@link #setup(ElectionContext)} to ensure
+ * the election process is init'd. Next call
+ * {@link #joinElection(ElectionContext)} to start the leader election.
+ *
+ * The implementation follows the classic ZooKeeper recipe of creating an
+ * ephemeral, sequential node for each candidate and then looking at the set
+ * of such nodes - if the created node is the lowest sequential node, the
+ * candidate that created the node is the leader. If not, the candidate puts
+ * a watch on the next lowest node it finds, and if that node goes down,
+ * starts the whole process over by checking if it's the lowest sequential node, etc.
+ *
+ * TODO: now we could just reuse the lock package code for leader election
+ */
+public class LeaderElector {
+ private static Logger log = LoggerFactory.getLogger(LeaderElector.class);
+
+ private static final String ELECTION_NODE = "/election";
+
+ private final static Pattern LEADER_SEQ = Pattern.compile(".*?/?.*?-n_(\\d+)");
+ private final static Pattern SESSION_ID = Pattern.compile(".*?/?(.*?-.*?)-n_\\d+");
+
+ protected SolrZkClient zkClient;
+
+ private ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor();
+
+ public LeaderElector(SolrZkClient zkClient) {
+ this.zkClient = zkClient;
+ }
+
+ /**
+ * Check if the candidate with the given n_* sequence number is the leader.
+ * If it is, set the leaderId on the leader zk node. If it is not, start
+ * watching the candidate that is in line before this one - if it goes down, check
+ * if this candidate is the leader again.
+ * @param leaderSeqPath
+ *
+ * @param seq
+ * @param context
+ * @param replacement has someone else been the leader already?
+ * @throws KeeperException
+ * @throws InterruptedException
+ * @throws IOException
+ * @throws UnsupportedEncodingException
+ */
+ private void checkIfIamLeader(final String leaderSeqPath, final int seq, final ElectionContext context, boolean replacement) throws KeeperException,
+ InterruptedException, IOException {
+ // get all other numbers...
+ final String holdElectionPath = context.electionPath + ELECTION_NODE;
+ List seqs = zkClient.getChildren(holdElectionPath, null, true);
+
+ sortSeqs(seqs);
+ List intSeqs = getSeqs(seqs);
+ if (seq <= intSeqs.get(0)) {
+ runIamLeaderProcess(leaderSeqPath, context, replacement);
+ } else {
+ // I am not the leader - watch the node below me
+ int i = 1;
+ for (; i < intSeqs.size(); i++) {
+ int s = intSeqs.get(i);
+ if (seq < s) {
+ // we found who we come before - watch the guy in front
+ break;
+ }
+ }
+ int index = i - 2;
+ if (index < 0) {
+ log.warn("Our node is no longer in line to be leader");
+ return;
+ }
+ try {
+ zkClient.getData(holdElectionPath + "/" + seqs.get(index),
+ new Watcher() {
+
+ @Override
+ public void process(WatchedEvent event) {
+ // am I the next leader?
+ try {
+ checkIfIamLeader(leaderSeqPath, seq, context, true);
+ } catch (InterruptedException e) {
+ // Restore the interrupted status
+ Thread.currentThread().interrupt();
+ log.warn("", e);
+ } catch (IOException e) {
+ log.warn("", e);
+ } catch (Exception e) {
+ log.warn("", e);
+ }
+ }
+
+ }, null, true);
+ } catch (KeeperException.SessionExpiredException e) {
+ throw e;
+ } catch (KeeperException e) {
+ // we couldn't set our watch - the node before us may already be down?
+ // we need to check if we are the leader again
+ checkIfIamLeader(leaderSeqPath, seq, context, true);
+ }
+ }
+ }
+
+ // TODO: get this core param out of here
+ protected void runIamLeaderProcess(String leaderSeqPath, final ElectionContext context, boolean weAreReplacement) throws KeeperException,
+ InterruptedException, IOException {
+
+ context.runLeaderProcess(leaderSeqPath, weAreReplacement);
+ }
+
+ /**
+ * Returns int given String of form n_0000000001 or n_0000000003, etc.
+ *
+ * @param nStringSequence
+ * @return
+ */
+ private int getSeq(String nStringSequence) {
+ int seq = 0;
+ Matcher m = LEADER_SEQ.matcher(nStringSequence);
+ if (m.matches()) {
+ seq = Integer.parseInt(m.group(1));
+ } else {
+ throw new IllegalStateException("Could not find regex match in:"
+ + nStringSequence);
+ }
+ return seq;
+ }
+
+ private String getNodeId(String nStringSequence) {
+ String id;
+ Matcher m = SESSION_ID.matcher(nStringSequence);
+ if (m.matches()) {
+ id = m.group(1);
+ } else {
+ throw new IllegalStateException("Could not find regex match in:"
+ + nStringSequence);
+ }
+ return id;
+ }
+
+ /**
+ * Returns int list given list of form n_0000000001, n_0000000003, etc.
+ *
+ * @param seqs
+ * @return
+ */
+ private List getSeqs(List seqs) {
+ List intSeqs = new ArrayList(seqs.size());
+ for (String seq : seqs) {
+ intSeqs.add(getSeq(seq));
+ }
+ return intSeqs;
+ }
+
+ /**
+ * Begin participating in the election process. Gets a new sequential number
+ * and begins watching the node with the sequence number before it, unless it
+ * is the lowest number, in which case, initiates the leader process. If the
+ * node that is watched goes down, check if we are the new lowest node, else
+ * watch the next lowest numbered node.
+ *
+ * @param context
+ * @param SolrCore - optional - sometimes null
+ * @return sequential node number
+ * @throws KeeperException
+ * @throws InterruptedException
+ * @throws IOException
+ * @throws UnsupportedEncodingException
+ */
+ public int joinElection(ElectionContext context) throws KeeperException, InterruptedException, IOException {
+ final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
+
+ long sessionId = zkClient.getSolrZooKeeper().getSessionId();
+ String id = sessionId + "-" + context.id;
+ String leaderSeqPath = null;
+ boolean cont = true;
+ int tries = 0;
+ while (cont) {
+ try {
+ leaderSeqPath = zkClient.create(shardsElectZkPath + "/" + id + "-n_", null,
+ CreateMode.EPHEMERAL_SEQUENTIAL, false);
+ cont = false;
+ } catch (ConnectionLossException e) {
+ // we don't know if we made our node or not...
+ List entries = zkClient.getChildren(shardsElectZkPath, null, true);
+
+ boolean foundId = false;
+ for (String entry : entries) {
+ String nodeId = getNodeId(entry);
+ if (id.equals(nodeId)) {
+ // we did create our node...
+ foundId = true;
+ break;
+ }
+ }
+ if (!foundId) {
+ throw e;
+ }
+
+ } catch (KeeperException.NoNodeException e) {
+ // we must have failed in creating the election node - someone else must
+ // be working on it, lets try again
+ if (tries++ > 9) {
+ throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
+ "", e);
+ }
+ cont = true;
+ Thread.sleep(50);
+ }
+ }
+ int seq = getSeq(leaderSeqPath);
+ checkIfIamLeader(leaderSeqPath, seq, context, false);
+
+ return seq;
+ }
+
+ /**
+ * Set up any ZooKeeper nodes needed for leader election.
+ *
+ * @param shardId
+ * @param collection
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public void setup(final ElectionContext context) throws InterruptedException,
+ KeeperException {
+ String electZKPath = context.electionPath + LeaderElector.ELECTION_NODE;
+
+ zkCmdExecutor.ensureExists(electZKPath, zkClient);
+ }
+
+ /**
+ * Sort n string sequence list.
+ *
+ * @param seqs
+ */
+ private void sortSeqs(List seqs) {
+ Collections.sort(seqs, new Comparator() {
+
+ @Override
+ public int compare(String o1, String o2) {
+ return Integer.valueOf(getSeq(o1)).compareTo(
+ Integer.valueOf(getSeq(o2)));
+ }
+ });
+ }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/NodeStateWatcher.java b/solr/core/src/java/org/apache/solr/cloud/NodeStateWatcher.java
new file mode 100644
index 00000000000..b5bd1fc57c5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/NodeStateWatcher.java
@@ -0,0 +1,124 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.solr.common.cloud.CoreState;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Watcher for node state changes.
+ */
+public class NodeStateWatcher implements Watcher {
+
+ private static Logger log = LoggerFactory.getLogger(NodeStateWatcher.class);
+
+ public static interface NodeStateChangeListener {
+ void coreChanged(String nodeName, Set states)
+ throws KeeperException, InterruptedException;
+ }
+
+ private final SolrZkClient zkClient;
+ private final String path;
+ private volatile Set currentState = new HashSet();
+ private final NodeStateChangeListener listener;
+ private final String nodeName;
+
+
+ public Set getCurrentState() {
+ return currentState;
+ }
+
+ public NodeStateWatcher(SolrZkClient zkClient, String nodeName, String path,
+ NodeStateChangeListener listener) throws KeeperException, InterruptedException {
+ this.nodeName = nodeName;
+ this.zkClient = zkClient;
+ this.path = path;
+ this.listener = listener;
+ processStateChange();
+ }
+
+ @Override
+ public void process(WatchedEvent event) {
+ try {
+ processStateChange();
+ } catch (InterruptedException e) {
+ // Restore the interrupted status
+ Thread.currentThread().interrupt();
+ return;
+ } catch (Exception e) {
+ log.warn("Error processing state change", e);
+ }
+ }
+
+ private void processStateChange() throws KeeperException, InterruptedException {
+ byte[] data = zkClient.getData(path, this, null, true);
+
+ if (data != null) {
+ CoreState[] states = CoreState.load(data);
+ List stateList = Arrays.asList(states);
+ HashSet modifiedCores = new HashSet();
+ modifiedCores.addAll(stateList);
+ modifiedCores.removeAll(currentState);
+
+ HashSet newState = new HashSet();
+ newState.addAll(stateList);
+
+ HashMap lookup = new HashMap();
+ for(CoreState state: states) {
+ lookup.put(state.getCoreName(), state);
+ }
+
+ //check for status change
+ for(CoreState state: currentState) {
+ if(lookup.containsKey(state.getCoreName())) {
+ if(!state.getProperties().equals(lookup.get(state.getCoreName()).getProperties())) {
+ modifiedCores.add(lookup.get(state.getCoreName()));
+ }
+ }
+ }
+
+ currentState = Collections.unmodifiableSet(newState);
+
+ if (modifiedCores.size() > 0) {
+ try {
+ listener.coreChanged(nodeName, Collections.unmodifiableSet(modifiedCores));
+ } catch (KeeperException e) {
+ log.warn("Could not talk to ZK", e);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ log.warn("Could not talk to ZK", e);
+ }
+ }
+
+ } else {
+ // ignore null state
+ }
+ }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
new file mode 100644
index 00000000000..357e04b4619
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -0,0 +1,450 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.solr.cloud.NodeStateWatcher.NodeStateChangeListener;
+import org.apache.solr.cloud.ShardLeaderWatcher.ShardLeaderListener;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.CoreState;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkOperation;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Cluster leader. Responsible node assignments, cluster state file?
+ */
+public class Overseer implements NodeStateChangeListener, ShardLeaderListener {
+
+ public static final String ASSIGNMENTS_NODE = "/node_assignments";
+ public static final String STATES_NODE = "/node_states";
+ private static Logger log = LoggerFactory.getLogger(Overseer.class);
+
+ private final SolrZkClient zkClient;
+ private final ZkStateReader reader;
+
+ // node stateWatches
+ private HashMap nodeStateWatches = new HashMap();
+
+ // shard leader watchers (collection->slice->watcher)
+ private HashMap> shardLeaderWatches = new HashMap>();
+ private ZkCmdExecutor zkCmdExecutor;
+
+ public Overseer(final SolrZkClient zkClient, final ZkStateReader reader) throws KeeperException, InterruptedException {
+ log.info("Constructing new Overseer");
+ this.zkClient = zkClient;
+ this.zkCmdExecutor = new ZkCmdExecutor();
+ this.reader = reader;
+ createWatches();
+ }
+
+ public synchronized void createWatches()
+ throws KeeperException, InterruptedException {
+ addCollectionsWatch();
+ addLiveNodesWatch();
+ }
+
+ /*
+ * Watch for collections so we can add watches for its shard leaders.
+ */
+ private void addCollectionsWatch() throws KeeperException,
+ InterruptedException {
+
+ zkCmdExecutor.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE, zkClient);
+
+ List collections = zkClient.getChildren(ZkStateReader.COLLECTIONS_ZKNODE, new Watcher(){
+ @Override
+ public void process(WatchedEvent event) {
+ try {
+ List collections = zkClient.getChildren(ZkStateReader.COLLECTIONS_ZKNODE, this, true);
+ collectionsChanged(collections);
+ } catch (KeeperException e) {
+ if (e.code() == Code.CONNECTIONLOSS || e.code() == Code.SESSIONEXPIRED) {
+ log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
+ return;
+ }
+ } catch (InterruptedException e) {
+ // Restore the interrupted status
+ Thread.currentThread().interrupt();
+ log.warn("", e);
+ }
+ }
+ }, true);
+
+ collectionsChanged(collections);
+ }
+
+ private void collectionsChanged(Collection collections) throws KeeperException, InterruptedException {
+ synchronized (shardLeaderWatches) {
+ for(String collection: collections) {
+ if(!shardLeaderWatches.containsKey(collection)) {
+ shardLeaderWatches.put(collection, new HashMap());
+ addShardLeadersWatch(collection);
+ }
+ }
+ //XXX not handling delete collections..
+ }
+ }
+
+ /**
+ * Add a watch for node containing shard leaders for a collection
+ * @param collection
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ private void addShardLeadersWatch(final String collection) throws KeeperException,
+ InterruptedException {
+
+ zkCmdExecutor.ensureExists(ZkStateReader.getShardLeadersPath(collection, null), zkClient);
+
+ final List leaderNodes = zkClient.getChildren(
+ ZkStateReader.getShardLeadersPath(collection, null), new Watcher() {
+
+ @Override
+ public void process(WatchedEvent event) {
+ try {
+ List leaderNodes = zkClient.getChildren(
+ ZkStateReader.getShardLeadersPath(collection, null), this, true);
+
+ processLeaderNodesChanged(collection, leaderNodes);
+ } catch (KeeperException e) {
+ if (e.code() == KeeperException.Code.SESSIONEXPIRED
+ || e.code() == KeeperException.Code.CONNECTIONLOSS) {
+ log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
+ return;
+ }
+ SolrException.log(log, "", e);
+ throw new ZooKeeperException(
+ SolrException.ErrorCode.SERVER_ERROR, "", e);
+ } catch (InterruptedException e) {
+ // Restore the interrupted status
+ Thread.currentThread().interrupt();
+ }
+ }
+ }, true);
+
+ processLeaderNodesChanged(collection, leaderNodes);
+ }
+
+ /**
+ * Process change in shard leaders. Make sure we have watches for each leader.
+ */
+ private void processLeaderNodesChanged(final String collection, final Collection shardIds) {
+ if(log.isInfoEnabled()) {
+ log.info("Leader nodes changed for collection: " + collection + " nodes now:" + shardIds);
+ }
+
+ Map watches = shardLeaderWatches.get(collection);
+ Set currentWatches = new HashSet();
+ currentWatches.addAll(watches.keySet());
+
+ Set newLeaders = complement(shardIds, currentWatches);
+
+ Set lostLeaders = complement(currentWatches, shardIds);
+ //remove watches for lost shards
+ for (String shardId : lostLeaders) {
+ ShardLeaderWatcher watcher = watches.remove(shardId);
+ if (watcher != null) {
+ watcher.close();
+ announceLeader(collection, shardId, new ZkCoreNodeProps(new ZkNodeProps())); //removes loeader for shard
+ }
+ }
+
+ //add watches for the new shards
+ for(String shardId: newLeaders) {
+ try {
+ ShardLeaderWatcher watcher = new ShardLeaderWatcher(shardId, collection, zkClient, this);
+ watches.put(shardId, watcher);
+ } catch (KeeperException e) {
+ log.error("Failed to create watcher for shard leader col:" + collection + " shard:" + shardId + ", exception: " + e.getClass());
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ log.error("Failed to create watcher for shard leader col:" + collection + " shard:" + shardId + ", exception: " + e.getClass());
+ }
+ }
+ }
+
+ private void addLiveNodesWatch() throws KeeperException,
+ InterruptedException {
+ List liveNodes = zkCmdExecutor.retryOperation(new ZkOperation() {
+
+ @Override
+ public Object execute() throws KeeperException, InterruptedException {
+ return zkClient.getChildren(
+ ZkStateReader.LIVE_NODES_ZKNODE, new Watcher() {
+
+ @Override
+ public void process(WatchedEvent event) {
+ try {
+ List liveNodes = zkClient.getChildren(
+ ZkStateReader.LIVE_NODES_ZKNODE, this, true);
+ Set liveNodesSet = new HashSet();
+ liveNodesSet.addAll(liveNodes);
+ processLiveNodesChanged(nodeStateWatches.keySet(), liveNodes);
+ } catch (KeeperException e) {
+ if (e.code() == KeeperException.Code.SESSIONEXPIRED
+ || e.code() == KeeperException.Code.CONNECTIONLOSS) {
+ log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
+ return;
+ }
+ SolrException.log(log, "", e);
+ throw new ZooKeeperException(
+ SolrException.ErrorCode.SERVER_ERROR, "", e);
+ } catch (InterruptedException e) {
+ // Restore the interrupted status
+ Thread.currentThread().interrupt();
+ }
+ }
+ }, true);
+ }
+ });
+
+ processLiveNodesChanged(Collections.emptySet(), liveNodes);
+ }
+
+ private void processLiveNodesChanged(Collection oldLiveNodes,
+ Collection liveNodes) throws InterruptedException, KeeperException {
+
+ Set upNodes = complement(liveNodes, oldLiveNodes);
+ if (upNodes.size() > 0) {
+ addNodeStateWatches(upNodes);
+ }
+
+ Set downNodes = complement(oldLiveNodes, liveNodes);
+ for(String node: downNodes) {
+ NodeStateWatcher watcher = nodeStateWatches.remove(node);
+ }
+ }
+
+ private void addNodeStateWatches(Set nodeNames) throws InterruptedException, KeeperException {
+
+ for (String nodeName : nodeNames) {
+ final String path = STATES_NODE + "/" + nodeName;
+ synchronized (nodeStateWatches) {
+ if (!nodeStateWatches.containsKey(nodeName)) {
+ zkCmdExecutor.ensureExists(path, zkClient);
+ nodeStateWatches.put(nodeName, new NodeStateWatcher(zkClient, nodeName, path, this));
+ } else {
+ log.debug("watch already added");
+ }
+ }
+ }
+ }
+
+ /**
+ * Try to assign core to the cluster
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ private CloudState updateState(CloudState state, String nodeName, CoreState coreState) throws KeeperException, InterruptedException {
+ String collection = coreState.getCollectionName();
+ String zkCoreNodeName = coreState.getCoreNodeName();
+
+ String shardId;
+ if (coreState.getProperties().get(ZkStateReader.SHARD_ID_PROP) == null) {
+ shardId = AssignShard.assignShard(collection, state);
+ } else {
+ shardId = coreState.getProperties().get(ZkStateReader.SHARD_ID_PROP);
+ }
+
+ Map props = new HashMap();
+ for (Entry entry : coreState.getProperties().entrySet()) {
+ props.put(entry.getKey(), entry.getValue());
+ }
+ ZkNodeProps zkProps = new ZkNodeProps(props);
+ Slice slice = state.getSlice(collection, shardId);
+ Map shardProps;
+ if (slice == null) {
+ shardProps = new HashMap();
+ } else {
+ shardProps = state.getSlice(collection, shardId).getShardsCopy();
+ }
+ shardProps.put(zkCoreNodeName, zkProps);
+
+ slice = new Slice(shardId, shardProps);
+ CloudState newCloudState = updateSlice(state, collection, slice);
+ return newCloudState;
+ }
+
+ private Set complement(Collection next,
+ Collection prev) {
+ Set downCollections = new HashSet();
+ downCollections.addAll(next);
+ downCollections.removeAll(prev);
+ return downCollections;
+ }
+
+ @Override
+ public void coreChanged(final String nodeName, final Set states) throws KeeperException, InterruptedException {
+ log.debug("Cores changed: " + nodeName + " states:" + states);
+ synchronized(reader.getUpdateLock()) {
+ reader.updateCloudState(true);
+ CloudState cloudState = reader.getCloudState();
+ for (CoreState state : states) {
+ cloudState = updateState(cloudState, nodeName, state);
+ }
+
+ try {
+ zkClient.setData(ZkStateReader.CLUSTER_STATE,
+ ZkStateReader.toJSON(cloudState), true);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+ "Interrupted while publishing new state", e);
+ }
+ }
+ }
+
+ public static void createClientNodes(SolrZkClient zkClient, String nodeName) throws KeeperException, InterruptedException {
+ final String node = STATES_NODE + "/" + nodeName;
+ if (log.isInfoEnabled()) {
+ log.info("creating node:" + node);
+ }
+
+ ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor();
+ zkCmdExecutor.ensureExists(node, zkClient);
+ }
+
+ private CloudState updateSlice(CloudState state, String collection, Slice slice) {
+
+ final Map> newStates = new LinkedHashMap>();
+ newStates.putAll(state.getCollectionStates());
+
+ if (!newStates.containsKey(collection)) {
+ newStates.put(collection, new LinkedHashMap());
+ }
+
+ final Map slices = newStates.get(collection);
+ if (!slices.containsKey(slice.getName())) {
+ slices.put(slice.getName(), slice);
+ } else {
+ final Map shards = new LinkedHashMap();
+ final Slice existingSlice = slices.get(slice.getName());
+ shards.putAll(existingSlice.getShards());
+ //XXX preserve existing leader
+ for(Entry edit: slice.getShards().entrySet()) {
+ if(existingSlice.getShards().get(edit.getKey())!=null && existingSlice.getShards().get(edit.getKey()).containsKey(ZkStateReader.LEADER_PROP)) {
+ HashMap newProps = new HashMap();
+ newProps.putAll(edit.getValue().getProperties());
+ newProps.put(ZkStateReader.LEADER_PROP, existingSlice.getShards().get(edit.getKey()).get(ZkStateReader.LEADER_PROP));
+ shards.put(edit.getKey(), new ZkNodeProps(newProps));
+ } else {
+ shards.put(edit.getKey(), edit.getValue());
+ }
+ }
+ final Slice updatedSlice = new Slice(slice.getName(), shards);
+ slices.put(slice.getName(), updatedSlice);
+ }
+ return new CloudState(state.getLiveNodes(), newStates);
+ }
+
+ private CloudState setShardLeader(CloudState state, String collection, String sliceName, String leaderUrl) {
+
+ boolean updated = false;
+ final Map> newStates = new LinkedHashMap>();
+ newStates.putAll(state.getCollectionStates());
+
+ final Map slices = newStates.get(collection);
+
+ if(slices==null) {
+ log.error("Could not mark shard leader for non existing collection.");
+ return state;
+ }
+
+ if (!slices.containsKey(sliceName)) {
+ log.error("Could not mark leader for non existing slice.");
+ return state;
+ } else {
+ final Map newShards = new LinkedHashMap();
+ for(Entry shard: slices.get(sliceName).getShards().entrySet()) {
+ Map newShardProps = new LinkedHashMap();
+ newShardProps.putAll(shard.getValue().getProperties());
+
+ String wasLeader = newShardProps.remove(ZkStateReader.LEADER_PROP); //clean any previously existed flag
+
+ ZkCoreNodeProps zkCoreNodeProps = new ZkCoreNodeProps(new ZkNodeProps(newShardProps));
+ if(leaderUrl!=null && leaderUrl.equals(zkCoreNodeProps.getCoreUrl())) {
+ newShardProps.put(ZkStateReader.LEADER_PROP,"true");
+ if (wasLeader == null) {
+ updated = true;
+ }
+ } else {
+ if (wasLeader != null) {
+ updated = true;
+ }
+ }
+ newShards.put(shard.getKey(), new ZkNodeProps(newShardProps));
+ }
+ Slice slice = new Slice(sliceName, newShards);
+ slices.put(sliceName, slice);
+ }
+ if (updated) {
+ return new CloudState(state.getLiveNodes(), newStates);
+ } else {
+ return state;
+ }
+ }
+
+ @Override
+ public void announceLeader(String collection, String shardId, ZkCoreNodeProps props) {
+ synchronized (reader.getUpdateLock()) {
+ try {
+ reader.updateCloudState(true); // get fresh copy of the state
+ final CloudState state = reader.getCloudState();
+ final CloudState newState = setShardLeader(state, collection, shardId,
+ props.getCoreUrl());
+ if (state != newState) { // if same instance was returned no need to
+ // update state
+ log.info("Announcing new leader: coll: " + collection + " shard: " + shardId + " props:" + props);
+ zkClient.setData(ZkStateReader.CLUSTER_STATE,
+ ZkStateReader.toJSON(newState), true);
+
+ } else {
+ log.debug("State was not changed.");
+ }
+ } catch (KeeperException e) {
+ log.warn("Could not announce new leader coll:" + collection + " shard:" + shardId + ", exception: " + e.getClass());
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ log.warn("Could not promote new leader coll:" + collection + " shard:" + shardId + ", exception: " + e.getClass());
+ }
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
new file mode 100644
index 00000000000..35d91496c4a
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -0,0 +1,272 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.PrepRecovery;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.RequestHandlers.LazyRequestHandlerWrapper;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.update.UpdateLog;
+import org.apache.solr.update.UpdateLog.RecoveryInfo;
+import org.apache.solr.util.RefCounted;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RecoveryStrategy extends Thread {
+ private static final int MAX_RETRIES = 100;
+ private static final int INTERRUPTED = 101;
+ private static final int START_TIMEOUT = 100;
+
+ private static final String REPLICATION_HANDLER = "/replication";
+
+ private static Logger log = LoggerFactory.getLogger(RecoveryStrategy.class);
+
+ private volatile boolean close = false;
+
+ private ZkController zkController;
+ private String baseUrl;
+ private String coreZkNodeName;
+ private ZkStateReader zkStateReader;
+ private volatile String coreName;
+ private int retries;
+ private SolrCore core;
+
+ public RecoveryStrategy(SolrCore core) {
+ this.core = core;
+ this.coreName = core.getName();
+
+ zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
+ zkStateReader = zkController.getZkStateReader();
+ baseUrl = zkController.getBaseUrl();
+ coreZkNodeName = zkController.getNodeName() + "_" + coreName;
+
+ }
+
+ // make sure any threads stop retrying
+ public void close() {
+ close = true;
+ interrupt();
+ }
+
+
+ private void recoveryFailed(final SolrCore core,
+ final ZkController zkController, final String baseUrl,
+ final String shardZkNodeName, final CoreDescriptor cd) {
+ SolrException.log(log, "Recovery failed - I give up.");
+ zkController.publishAsRecoveryFailed(baseUrl, cd,
+ shardZkNodeName, core.getName());
+ close = true;
+ }
+
+ private void replicate(String nodeName, SolrCore core, String shardZkNodeName, ZkNodeProps leaderprops, String baseUrl)
+ throws SolrServerException, IOException {
+ // start buffer updates to tran log
+ // and do recovery - either replay via realtime get (eventually)
+ // or full index replication
+
+ String leaderBaseUrl = leaderprops.get(ZkStateReader.BASE_URL_PROP);
+ ZkCoreNodeProps leaderCNodeProps = new ZkCoreNodeProps(leaderprops);
+ String leaderUrl = leaderCNodeProps.getCoreUrl();
+ String leaderCoreName = leaderCNodeProps.getCoreName();
+
+ log.info("Attempt to replicate from " + leaderUrl);
+
+ // if we are the leader, either we are trying to recover faster
+ // then our ephemeral timed out or we are the only node
+ if (!leaderBaseUrl.equals(baseUrl)) {
+
+ CommonsHttpSolrServer server = new CommonsHttpSolrServer(leaderBaseUrl);
+ server.setSoTimeout(15000);
+ PrepRecovery prepCmd = new PrepRecovery();
+ prepCmd.setCoreName(leaderCoreName);
+ prepCmd.setNodeName(nodeName);
+ prepCmd.setCoreNodeName(shardZkNodeName);
+
+ server.request(prepCmd);
+ server.shutdown();
+
+ // use rep handler directly, so we can do this sync rather than async
+ SolrRequestHandler handler = core.getRequestHandler(REPLICATION_HANDLER);
+ if (handler instanceof LazyRequestHandlerWrapper) {
+ handler = ((LazyRequestHandlerWrapper)handler).getWrappedHandler();
+ }
+ ReplicationHandler replicationHandler = (ReplicationHandler) handler;
+
+ if (replicationHandler == null) {
+ throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+ "Skipping recovery, no " + REPLICATION_HANDLER + " handler found");
+ }
+
+ ModifiableSolrParams solrParams = new ModifiableSolrParams();
+ solrParams.set(ReplicationHandler.MASTER_URL, leaderUrl + "replication");
+
+ if (close) retries = INTERRUPTED;
+ boolean success = replicationHandler.doFetch(solrParams, true); // TODO: look into making sure fore=true does not download files we already have
+
+ if (!success) {
+ throw new SolrException(ErrorCode.SERVER_ERROR, "Replication for recovery failed.");
+ }
+
+ // solrcloud_debug
+// try {
+// RefCounted searchHolder = core.getNewestSearcher(false);
+// SolrIndexSearcher searcher = searchHolder.get();
+// try {
+// System.out.println(core.getCoreDescriptor().getCoreContainer().getZkController().getNodeName() + " replicated "
+// + searcher.search(new MatchAllDocsQuery(), 1).totalHits + " from " + leaderUrl + " gen:" + core.getDeletionPolicy().getLatestCommit().getGeneration() + " data:" + core.getDataDir());
+// } finally {
+// searchHolder.decref();
+// }
+// } catch (Exception e) {
+//
+// }
+ }
+ }
+
+ @Override
+ public void run() {
+ boolean replayed = false;
+ boolean succesfulRecovery = false;
+
+ while (!succesfulRecovery && !close && !isInterrupted()) {
+ UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
+ if (ulog == null) return;
+
+ ulog.bufferUpdates();
+ replayed = false;
+ CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
+ try {
+ zkController.publish(core, ZkStateReader.RECOVERING);
+
+ ZkNodeProps leaderprops = zkStateReader.getLeaderProps(
+ cloudDesc.getCollectionName(), cloudDesc.getShardId());
+
+ // System.out.println("recover " + shardZkNodeName + " against " +
+ // leaderprops);
+ replicate(zkController.getNodeName(), core, coreZkNodeName,
+ leaderprops, ZkCoreNodeProps.getCoreUrl(baseUrl, coreName));
+
+ replay(ulog);
+ replayed = true;
+
+ // if there are pending recovery requests, don't advert as active
+ zkController.publishAsActive(baseUrl, core.getCoreDescriptor(), coreZkNodeName,
+ coreName);
+
+ succesfulRecovery = true;
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ log.warn("Recovery was interrupted", e);
+ retries = INTERRUPTED;
+ } catch (Throwable t) {
+ SolrException.log(log, "Error while trying to recover", t);
+ } finally {
+ if (!replayed) {
+ try {
+ ulog.dropBufferedUpdates();
+ } catch (Throwable t) {
+ SolrException.log(log, "", t);
+ }
+ }
+
+ }
+
+ if (!succesfulRecovery) {
+ // lets pause for a moment and we need to try again...
+ // TODO: we don't want to retry for some problems?
+ // Or do a fall off retry...
+ try {
+
+ SolrException.log(log, "Recovery failed - trying again...");
+ retries++;
+ if (retries >= MAX_RETRIES) {
+ if (retries == INTERRUPTED) {
+
+ } else {
+ // TODO: for now, give up after 10 tries - should we do more?
+ recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
+ core.getCoreDescriptor());
+ }
+ break;
+ }
+
+ } catch (Exception e) {
+ SolrException.log(log, "", e);
+ }
+
+ try {
+ Thread.sleep(Math.min(START_TIMEOUT * retries, 60000));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ log.warn("Recovery was interrupted", e);
+ retries = INTERRUPTED;
+ }
+ }
+
+ log.info("Finished recovery process");
+
+ }
+ }
+
+ private Future replay(UpdateLog ulog)
+ throws InterruptedException, ExecutionException, TimeoutException {
+ Future future = ulog.applyBufferedUpdates();
+ if (future == null) {
+ // no replay needed\
+ log.info("No replay needed");
+ } else {
+ // wait for replay
+ future.get();
+ }
+
+ // solrcloud_debug
+// try {
+// RefCounted searchHolder = core.getNewestSearcher(false);
+// SolrIndexSearcher searcher = searchHolder.get();
+// try {
+// System.out.println(core.getCoreDescriptor().getCoreContainer().getZkController().getNodeName() + " replayed "
+// + searcher.search(new MatchAllDocsQuery(), 1).totalHits);
+// } finally {
+// searchHolder.decref();
+// }
+// } catch (Exception e) {
+//
+// }
+
+ return future;
+ }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderWatcher.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderWatcher.java
new file mode 100644
index 00000000000..d15d1170920
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderWatcher.java
@@ -0,0 +1,90 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.KeeperException.Code;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A watcher for shard leader.
+ */
+public class ShardLeaderWatcher implements Watcher {
+ private static Logger logger = LoggerFactory.getLogger(ShardLeaderWatcher.class);
+ static interface ShardLeaderListener {
+ void announceLeader(String collection, String shardId, ZkCoreNodeProps props);
+ }
+
+ private final String shard;
+ private final String collection;
+ private final String path;
+ private final SolrZkClient zkClient;
+ private volatile boolean closed = false;
+ private final ShardLeaderListener listener;
+
+ public ShardLeaderWatcher(String shard, String collection,
+ SolrZkClient zkClient, ShardLeaderListener listener) throws KeeperException, InterruptedException {
+ this.shard = shard;
+ this.collection = collection;
+ this.path = ZkStateReader.getShardLeadersPath(collection, shard);
+ this.zkClient = zkClient;
+ this.listener = listener;
+ processLeaderChange();
+ }
+
+ private void processLeaderChange() throws KeeperException, InterruptedException {
+ if(closed) return;
+ try {
+ byte[] data = zkClient.getData(path, this, null, true);
+ if (data != null) {
+ final ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(ZkNodeProps.load(data));
+ listener.announceLeader(collection, shard, leaderProps);
+ }
+ } catch (KeeperException ke) {
+ //check if we lost connection or the node was gone
+ if (ke.code() != Code.CONNECTIONLOSS && ke.code() != Code.SESSIONEXPIRED
+ && ke.code() != Code.NONODE) {
+ throw ke;
+ }
+ }
+ }
+
+ @Override
+ public void process(WatchedEvent event) {
+ try {
+ processLeaderChange();
+ } catch (KeeperException e) {
+ logger.warn("Shard leader watch triggered but Solr cannot talk to zk.");
+ } catch (InterruptedException e) {
+ Thread.interrupted();
+ logger.warn("Shard leader watch triggered but Solr cannot talk to zk.");
+ }
+ }
+
+ public void close() {
+ closed = true;
+ }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
new file mode 100644
index 00000000000..694f7e93360
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
@@ -0,0 +1,227 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.update.PeerSync;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncStrategy {
+ protected final Logger log = LoggerFactory.getLogger(getClass());
+
+ public boolean sync(ZkController zkController, SolrCore core,
+ ZkNodeProps leaderProps) {
+ zkController.publish(core, ZkStateReader.SYNC);
+
+ // solrcloud_debug
+ // System.out.println("SYNC UP");
+ boolean success = syncReplicas(zkController, core, leaderProps);
+ return success;
+ }
+
+ private boolean syncReplicas(ZkController zkController, SolrCore core,
+ ZkNodeProps leaderProps) {
+ boolean success = false;
+ CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
+ String collection = cloudDesc.getCollectionName();
+ String shardId = cloudDesc.getShardId();
+
+ // first sync ourselves - we are the potential leader after all
+ try {
+ success = syncWithReplicas(zkController, core, leaderProps, collection,
+ shardId);
+ } catch (Exception e) {
+ SolrException.log(log, "Sync Failed", e);
+ }
+ try {
+ // if !success but no one else is in active mode,
+ // we are the leader anyway
+ // TODO: should we also be leader if there is only one other active?
+ // if we couldn't sync with it, it shouldn't be able to sync with us
+ if (!success
+ && !areAnyOtherReplicasActive(zkController, leaderProps, collection,
+ shardId)) {
+// System.out
+// .println("wasnt a success but no on else i active! I am the leader");
+
+ success = true;
+ }
+
+ if (success) {
+ // solrcloud_debug
+ // System.out.println("Sync success");
+ // we are the leader - tell all of our replias to sync with us
+
+ syncToMe(zkController, collection, shardId, leaderProps);
+
+ } else {
+
+ // solrcloud_debug
+ // System.out.println("Sync failure");
+ }
+
+ } catch (Exception e) {
+ SolrException.log(log, "Sync Failed", e);
+ }
+
+ return success;
+ }
+
+ private boolean areAnyOtherReplicasActive(ZkController zkController,
+ ZkNodeProps leaderProps, String collection, String shardId) {
+ CloudState cloudState = zkController.getZkStateReader().getCloudState();
+ Map slices = cloudState.getSlices(collection);
+ Slice slice = slices.get(shardId);
+ Map shards = slice.getShards();
+ for (Map.Entry shard : shards.entrySet()) {
+ String state = shard.getValue().get(ZkStateReader.STATE_PROP);
+// System.out.println("state:"
+// + state
+// + shard.getValue().get(ZkStateReader.NODE_NAME_PROP)
+// + " live: "
+// + cloudState.liveNodesContain(shard.getValue().get(
+// ZkStateReader.NODE_NAME_PROP)));
+ if ((state.equals(ZkStateReader.ACTIVE))
+ && cloudState.liveNodesContain(shard.getValue().get(
+ ZkStateReader.NODE_NAME_PROP))
+ && !new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals(
+ new ZkCoreNodeProps(leaderProps).getCoreUrl())) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ private boolean syncWithReplicas(ZkController zkController, SolrCore core,
+ ZkNodeProps props, String collection, String shardId)
+ throws MalformedURLException, SolrServerException, IOException {
+ List nodes = zkController.getZkStateReader()
+ .getReplicaProps(collection, shardId,
+ props.get(ZkStateReader.NODE_NAME_PROP),
+ props.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.ACTIVE); // TODO:
+ // should
+ // there
+ // be a
+ // state
+ // filter?
+
+ if (nodes == null) {
+ // I have no replicas
+ return true;
+ }
+
+ List syncWith = new ArrayList();
+ for (ZkCoreNodeProps node : nodes) {
+ // if we see a leader, must be stale state, and this is the guy that went down
+ if (!node.getNodeProps().keySet().contains(ZkStateReader.LEADER_PROP)) {
+ syncWith.add(node.getCoreUrl());
+ }
+ }
+
+
+ PeerSync peerSync = new PeerSync(core, syncWith, 1000);
+ return peerSync.sync();
+ }
+
+ private void syncToMe(ZkController zkController, String collection,
+ String shardId, ZkNodeProps leaderProps) throws MalformedURLException,
+ SolrServerException, IOException {
+
+ // sync everyone else
+ // TODO: we should do this in parallel at least
+ List nodes = zkController
+ .getZkStateReader()
+ .getReplicaProps(collection, shardId,
+ leaderProps.get(ZkStateReader.NODE_NAME_PROP),
+ leaderProps.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.ACTIVE);
+ if (nodes == null) {
+ // System.out.println("I have no replicas");
+ // I have no replicas
+ return;
+ }
+ //System.out.println("tell my replicas to sync");
+ ZkCoreNodeProps zkLeader = new ZkCoreNodeProps(leaderProps);
+ for (ZkCoreNodeProps node : nodes) {
+ try {
+ // TODO: do we first everyone register as sync phase? get the overseer
+ // to do it?
+ // TODO: this should be done in parallel
+ QueryRequest qr = new QueryRequest(params("qt", "/get", "getVersions",
+ Integer.toString(1000), "sync", zkLeader.getCoreUrl(), "distrib",
+ "false"));
+ CommonsHttpSolrServer server = new CommonsHttpSolrServer(
+ node.getCoreUrl());
+ //System.out.println("ask " + node.getCoreUrl() + " to sync");
+ NamedList rsp = server.request(qr);
+ //System.out.println("response about syncing to leader:" + rsp + " node:"
+ // + node.getCoreUrl() + " me:" + zkController.getBaseUrl());
+ boolean success = (Boolean) rsp.get("sync");
+ //System.out.println("success:" + success);
+ if (!success) {
+ // System.out
+ // .println("try and ask " + node.getCoreUrl() + " to recover");
+ log.info("try and ask " + node.getCoreUrl() + " to recover");
+ try {
+ server = new CommonsHttpSolrServer(node.getBaseUrl());
+ server.setSoTimeout(5000);
+ server.setConnectionTimeout(5000);
+
+ RequestRecovery recoverRequestCmd = new RequestRecovery();
+ recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
+ recoverRequestCmd.setCoreName(node.getCoreName());
+
+ server.request(recoverRequestCmd);
+ } catch (Exception e) {
+ log.info("Could not tell a replica to recover", e);
+ }
+ }
+ } catch (Exception e) {
+ SolrException.log(log, "Error syncing replica to leader", e);
+ }
+ }
+ }
+
+ public static ModifiableSolrParams params(String... params) {
+ ModifiableSolrParams msp = new ModifiableSolrParams();
+ for (int i = 0; i < params.length; i += 2) {
+ msp.add(params[i], params[i + 1]);
+ }
+ return msp;
+ }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 45fa0104025..80bb9e9ad09 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -20,24 +20,36 @@ package org.apache.solr.cloud;
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.CoreState;
import org.apache.solr.common.cloud.OnReconnect;
import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.update.UpdateLog;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -61,7 +73,7 @@ public final class ZkController {
private final static Pattern URL_POST = Pattern.compile("https?://(.*)");
private final static Pattern URL_PREFIX = Pattern.compile("(https?://).*");
-
+
// package private for tests
static final String CONFIGS_ZKNODE = "/configs";
@@ -69,10 +81,14 @@ public final class ZkController {
public final static String COLLECTION_PARAM_PREFIX="collection.";
public final static String CONFIGNAME_PROP="configName";
- private SolrZkClient zkClient;
+ private final Map coreStates = Collections.synchronizedMap(new HashMap());
+ private SolrZkClient zkClient;
+ private ZkCmdExecutor cmdExecutor;
private ZkStateReader zkStateReader;
+ private LeaderElector leaderElector;
+
private String zkServerAddress;
private String localHostPort;
@@ -82,20 +98,61 @@ public final class ZkController {
private String hostName;
+ private LeaderElector overseerElector;
+
+ private boolean SKIP_AUTO_RECOVERY = Boolean.getBoolean("solrcloud.skip.autorecovery");
+
+ // this can be null in which case recovery will be inactive
+ private CoreContainer cc;
+
+ public static void main(String[] args) throws Exception {
+ // start up a tmp zk server first
+ String zkServerAddress = args[0];
+
+ String solrHome = args[1];
+ String solrPort = args[2];
+
+ String confDir = args[3];
+ String confName = args[4];
+
+ SolrZkServer zkServer = new SolrZkServer("true", null, solrHome, solrPort);
+ zkServer.parseConfig();
+ zkServer.start();
+
+ SolrZkClient zkClient = new SolrZkClient(zkServerAddress, 15000, 5000,
+ new OnReconnect() {
+ @Override
+ public void command() {
+ }});
+
+ uploadConfigDir(zkClient, new File(confDir), confName);
+
+ zkServer.stop();
+ }
+
+
/**
- * @param zkServerAddress ZooKeeper server host address
+ * @param coreContainer if null, recovery will not be enabled
+ * @param zkServerAddress
* @param zkClientTimeout
* @param zkClientConnectTimeout
* @param localHost
* @param locaHostPort
* @param localHostContext
+ * @param numShards
* @throws InterruptedException
* @throws TimeoutException
* @throws IOException
*/
- public ZkController(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
- String localHostContext) throws InterruptedException,
+ public ZkController(CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
+ String localHostContext, final CurrentCoreDescriptorProvider registerOnReconnect) throws InterruptedException,
TimeoutException, IOException {
+ this.cc = cc;
+ if (localHostContext.contains("/")) {
+ throw new IllegalArgumentException("localHostContext ("
+ + localHostContext + ") should not contain a /");
+ }
+
this.zkServerAddress = zkServerAddress;
this.localHostPort = locaHostPort;
this.localHostContext = localHostContext;
@@ -107,67 +164,60 @@ public final class ZkController {
public void command() {
try {
- zkStateReader.makeCollectionsNodeWatches();
- zkStateReader.makeShardsWatches(true);
+ // we need to create all of our lost watches
+
+ // seems we dont need to do this again...
+ //Overseer.createClientNodes(zkClient, getNodeName());
+
+ ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient, zkStateReader);
+ overseerElector.joinElection(context);
+ zkStateReader.createClusterStateWatchersAndUpdate();
+
+ List descriptors = registerOnReconnect
+ .getCurrentDescriptors();
+ if (descriptors != null) {
+ // before registering as live, make sure everyone is in a
+ // recovery state
+ for (CoreDescriptor descriptor : descriptors) {
+ final String shardZkNodeName = getNodeName() + "_"
+ + descriptor.getName();
+ publishAsDown(getBaseUrl(), descriptor, shardZkNodeName,
+ descriptor.getName());
+ }
+ }
+
+ // we have to register as live first to pick up docs in the buffer
createEphemeralLiveNode();
- zkStateReader.updateCloudState(false);
- } catch (KeeperException e) {
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
+
+ // re register all descriptors
+ if (descriptors != null) {
+ for (CoreDescriptor descriptor : descriptors) {
+ // TODO: we need to think carefully about what happens when it was
+ // a leader that was expired - as well as what to do about leaders/overseers
+ // with connection loss
+ register(descriptor.getName(), descriptor, true);
+ }
+ }
+
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- } catch (IOException e) {
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
+ throw new ZooKeeperException(
+ SolrException.ErrorCode.SERVER_ERROR, "", e);
+ } catch (Exception e) {
+ SolrException.log(log, "", e);
+ throw new ZooKeeperException(
+ SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
});
+ cmdExecutor = new ZkCmdExecutor();
+ leaderElector = new LeaderElector(zkClient);
zkStateReader = new ZkStateReader(zkClient);
init();
}
- /**
- * @param shardId
- * @param collection
- * @throws IOException
- * @throws InterruptedException
- * @throws KeeperException
- */
- private void addZkShardsNode(String shardId, String collection) throws IOException, InterruptedException, KeeperException {
-
- String shardsZkPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + ZkStateReader.SHARDS_ZKNODE + "/" + shardId;
-
- try {
-
- // shards node
- if (!zkClient.exists(shardsZkPath)) {
- if (log.isInfoEnabled()) {
- log.info("creating zk shards node:" + shardsZkPath);
- }
- // makes shards zkNode if it doesn't exist
- zkClient.makePath(shardsZkPath, CreateMode.PERSISTENT, null);
-
- // TODO: consider how these notifications are being done
- // ping that there is a new shardId
- zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
-
- }
- } catch (KeeperException e) {
- // its okay if another beats us creating the node
- if (e.code() != KeeperException.Code.NODEEXISTS) {
- throw e;
- }
- }
-
- }
-
/**
* Closes the underlying ZooKeeper client.
*/
@@ -177,7 +227,7 @@ public final class ZkController {
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
- log.error("", e);
+ log.warn("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
@@ -192,7 +242,7 @@ public final class ZkController {
*/
public boolean configFileExists(String collection, String fileName)
throws KeeperException, InterruptedException {
- Stat stat = zkClient.exists(CONFIGS_ZKNODE + "/" + collection + "/" + fileName, null);
+ Stat stat = zkClient.exists(CONFIGS_ZKNODE + "/" + collection + "/" + fileName, null, true);
return stat != null;
}
@@ -213,7 +263,7 @@ public final class ZkController {
public byte[] getConfigFileData(String zkConfigName, String fileName)
throws KeeperException, InterruptedException {
String zkPath = CONFIGS_ZKNODE + "/" + zkConfigName + "/" + fileName;
- byte[] bytes = zkClient.getData(zkPath, null, null);
+ byte[] bytes = zkClient.getData(zkPath, null, null, true);
if (bytes == null) {
log.error("Config file contains no data:" + zkPath);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
@@ -271,20 +321,17 @@ public final class ZkController {
}
// makes nodes zkNode
- try {
- zkClient.makePath(ZkStateReader.LIVE_NODES_ZKNODE);
- } catch (KeeperException e) {
- // its okay if another beats us creating the node
- if (e.code() != KeeperException.Code.NODEEXISTS) {
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- }
- }
+ cmdExecutor.ensureExists(ZkStateReader.LIVE_NODES_ZKNODE, zkClient);
+ Overseer.createClientNodes(zkClient, getNodeName());
createEphemeralLiveNode();
- setUpCollectionsNode();
- zkStateReader.makeCollectionsNodeWatches();
+ cmdExecutor.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE, zkClient);
+
+ overseerElector = new LeaderElector(zkClient);
+ ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient, zkStateReader);
+ overseerElector.setup(context);
+ overseerElector.joinElection(context);
+ zkStateReader.createClusterStateWatchersAndUpdate();
} catch (IOException e) {
log.error("", e);
@@ -303,53 +350,17 @@ public final class ZkController {
}
}
+
+ public boolean isConnected() {
+ return zkClient.isConnected();
+ }
private void createEphemeralLiveNode() throws KeeperException,
InterruptedException {
String nodeName = getNodeName();
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
log.info("Register node as live in ZooKeeper:" + nodePath);
- Watcher liveNodeWatcher = new Watcher() {
-
- public void process(WatchedEvent event) {
- try {
- log.info("Updating live nodes:" + zkClient);
- try {
- zkStateReader.updateLiveNodes();
- } finally {
- // re-make watch
-
- String path = event.getPath();
- if(path == null) {
- // on shutdown, it appears this can trigger with a null path
- log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
- return;
- }
- zkClient.getChildren(event.getPath(), this);
- }
- } catch (KeeperException e) {
- if(e.code() == KeeperException.Code.SESSIONEXPIRED || e.code() == KeeperException.Code.CONNECTIONLOSS) {
- log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
- return;
- }
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- } catch (InterruptedException e) {
- // Restore the interrupted status
- Thread.currentThread().interrupt();
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- } catch (IOException e) {
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- }
-
- }
-
- };
+
try {
boolean nodeDeleted = true;
try {
@@ -358,7 +369,7 @@ public final class ZkController {
// until expiration timeout - so a node won't be created here because
// it exists, but eventually the node will be removed. So delete
// in case it exists and create a new node.
- zkClient.delete(nodePath, -1);
+ zkClient.delete(nodePath, -1, true);
} catch (KeeperException.NoNodeException e) {
// fine if there is nothing to delete
// TODO: annoying that ZK logs a warning on us
@@ -369,25 +380,17 @@ public final class ZkController {
.info("Found a previous node that still exists while trying to register a new live node "
+ nodePath + " - removing existing node to create another.");
}
- zkClient.makePath(nodePath, CreateMode.EPHEMERAL);
+ zkClient.makePath(nodePath, CreateMode.EPHEMERAL, true);
} catch (KeeperException e) {
// its okay if the node already exists
if (e.code() != KeeperException.Code.NODEEXISTS) {
throw e;
}
- }
- zkClient.getChildren(ZkStateReader.LIVE_NODES_ZKNODE, liveNodeWatcher);
- try {
- zkStateReader.updateLiveNodes();
- } catch (IOException e) {
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- }
+ }
}
public String getNodeName() {
- return hostName + ":" + localHostPort + "_"+ localHostContext;
+ return hostName + ":" + localHostPort + "_" + localHostContext;
}
/**
@@ -398,7 +401,7 @@ public final class ZkController {
*/
public boolean pathExists(String path) throws KeeperException,
InterruptedException {
- return zkClient.exists(path);
+ return zkClient.exists(path, true);
}
/**
@@ -417,15 +420,14 @@ public final class ZkController {
if (log.isInfoEnabled()) {
log.info("Load collection config from:" + path);
}
- byte[] data = zkClient.getData(path, null, null);
- ZkNodeProps props = new ZkNodeProps();
+ byte[] data = zkClient.getData(path, null, null, true);
if(data != null) {
- props.load(data);
+ ZkNodeProps props = ZkNodeProps.load(data);
configName = props.get(CONFIGNAME_PROP);
}
- if (configName != null && !zkClient.exists(CONFIGS_ZKNODE + "/" + configName)) {
+ if (configName != null && !zkClient.exists(CONFIGS_ZKNODE + "/" + configName, true)) {
log.error("Specified config does not exist in ZooKeeper:" + configName);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"Specified config does not exist in ZooKeeper:" + configName);
@@ -434,67 +436,224 @@ public final class ZkController {
return configName;
}
+
/**
* Register shard with ZooKeeper.
*
* @param coreName
* @param cloudDesc
- * @param forcePropsUpdate update solr.xml core props even if the shard is already registered
- * @throws IOException
- * @throws KeeperException
- * @throws InterruptedException
+ * @return
+ * @throws Exception
*/
- public void register(String coreName, CloudDescriptor cloudDesc, boolean forcePropsUpdate) throws IOException,
- KeeperException, InterruptedException {
- String shardUrl = localHostName + ":" + localHostPort + "/" + localHostContext
- + "/" + coreName;
-
- String collection = cloudDesc.getCollectionName();
-
- String shardsZkPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + ZkStateReader.SHARDS_ZKNODE + "/" + cloudDesc.getShardId();
+ public String register(String coreName, final CoreDescriptor desc) throws Exception {
+ return register(coreName, desc, false);
+ }
+
- boolean shardZkNodeAlreadyExists = zkClient.exists(shardsZkPath);
+ /**
+ * Register shard with ZooKeeper.
+ *
+ * @param coreName
+ * @param desc
+ * @param recoverReloadedCores
+ * @return
+ * @throws Exception
+ */
+ public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores) throws Exception {
+ final String baseUrl = getBaseUrl();
- if(shardZkNodeAlreadyExists && !forcePropsUpdate) {
- return;
- }
+ final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
+ final String collection = cloudDesc.getCollectionName();
+
+ log.info("Attempting to update " + ZkStateReader.CLUSTER_STATE + " version "
+ + null);
+ CloudState state = CloudState.load(zkClient, zkStateReader.getCloudState().getLiveNodes());
+
+ final String coreZkNodeName = getNodeName() + "_" + coreName;
+
+ String shardId = cloudDesc.getShardId();
+
+ Map props = new HashMap();
+ props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
+ props.put(ZkStateReader.CORE_NAME_PROP, coreName);
+ props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+ props.put(ZkStateReader.ROLES_PROP, cloudDesc.getRoles());
+ props.put(ZkStateReader.STATE_PROP, ZkStateReader.DOWN);
+
if (log.isInfoEnabled()) {
- log.info("Register shard - core:" + coreName + " address:"
- + shardUrl);
+ log.info("Register shard - core:" + coreName + " address:"
+ + baseUrl + " shardId:" + shardId);
}
- ZkNodeProps props = new ZkNodeProps();
- props.put(ZkStateReader.URL_PROP, shardUrl);
+ // we only put a subset of props into the leader node
+ ZkNodeProps leaderProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
+ props.get(ZkStateReader.BASE_URL_PROP), ZkStateReader.CORE_NAME_PROP,
+ props.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.NODE_NAME_PROP,
+ props.get(ZkStateReader.NODE_NAME_PROP));
- props.put(ZkStateReader.NODE_NAME, getNodeName());
- byte[] bytes = props.store();
+ joinElection(collection, coreZkNodeName, shardId, leaderProps);
+
+ String leaderUrl = zkStateReader.getLeaderUrl(collection,
+ cloudDesc.getShardId(), 30000);
+
+ String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
+ log.info("We are " + ourUrl + " and leader is " + leaderUrl);
+ boolean isLeader = leaderUrl.equals(ourUrl);
- String shardZkNodeName = getNodeName() + "_" + coreName;
- if(shardZkNodeAlreadyExists && forcePropsUpdate) {
- zkClient.setData(shardsZkPath + "/" + shardZkNodeName, bytes);
- // tell everyone to update cloud info
- zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
- } else {
- addZkShardsNode(cloudDesc.getShardId(), collection);
+ SolrCore core = null;
+ if (cc != null) { // CoreContainer only null in tests
try {
- zkClient.create(shardsZkPath + "/" + shardZkNodeName, bytes,
- CreateMode.PERSISTENT);
- // tell everyone to update cloud info
- zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
- } catch (KeeperException e) {
- // its okay if the node already exists
- if (e.code() != KeeperException.Code.NODEEXISTS) {
- throw e;
+ core = cc.getCore(desc.getName());
+
+ boolean startRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
+ collection, coreZkNodeName, shardId, leaderProps, core, cc);
+ if (!startRecovery) {
+ publishAsActive(baseUrl, desc, coreZkNodeName, coreName);
}
- // for some reason the shard already exists, though it didn't when we
- // started registration - just return
- return;
+ } finally {
+ if (core != null) {
+ core.close();
+ }
+ }
+ } else {
+ publishAsActive(baseUrl, desc, coreZkNodeName, coreName);
+ }
+
+ // make sure we have an update cluster state right away
+ zkStateReader.updateCloudState(true);
+
+ return shardId;
+ }
+
+
+ private void joinElection(final String collection,
+ final String shardZkNodeName, String shardId, ZkNodeProps leaderProps) throws InterruptedException, KeeperException, IOException {
+ ElectionContext context = new ShardLeaderElectionContext(leaderElector, shardId,
+ collection, shardZkNodeName, leaderProps, this, cc);
+
+ leaderElector.setup(context);
+ leaderElector.joinElection(context);
+ }
+
+
+ private boolean checkRecovery(String coreName, final CoreDescriptor desc,
+ boolean recoverReloadedCores, final boolean isLeader,
+ final CloudDescriptor cloudDesc, final String collection,
+ final String shardZkNodeName, String shardId, ZkNodeProps leaderProps,
+ SolrCore core, CoreContainer cc) throws InterruptedException,
+ KeeperException, IOException, ExecutionException {
+
+
+ boolean doRecovery = true;
+
+
+ if (isLeader) {
+ doRecovery = false;
+
+ // recover from local transaction log and wait for it to complete before
+ // going active
+ // TODO: should this be moved to another thread? To recoveryStrat?
+ // TODO: should this actually be done earlier, before (or as part of)
+ // leader election perhaps?
+ // TODO: ensure that a replica that is trying to recover waits until I'm
+ // active (or don't make me the
+ // leader until my local replay is done. But this replay is only needed
+ // on the leader - replicas
+ // will do recovery anyway
+
+ UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
+ if (!core.isReloaded() && ulog != null) {
+ Future recoveryFuture = core.getUpdateHandler()
+ .getUpdateLog().recoverFromLog();
+ if (recoveryFuture != null) {
+ recoveryFuture.get(); // NOTE: this could potentially block for
+ // minutes or more!
+ // TODO: public as recovering in the mean time?
+ }
+ }
+ return false;
+ } else {
+
+ if (core.isReloaded() && !recoverReloadedCores) {
+ doRecovery = false;
}
}
+
+ if (doRecovery && !SKIP_AUTO_RECOVERY) {
+ log.info("Core needs to recover:" + core.getName());
+ core.getUpdateHandler().getSolrCoreState().doRecovery(core);
+ return true;
+ }
+
+ return false;
+ }
+
+ public String getBaseUrl() {
+ final String baseUrl = localHostName + ":" + localHostPort + "/" + localHostContext;
+ return baseUrl;
+ }
+
+
+ void publishAsActive(String shardUrl,
+ final CoreDescriptor cd, String shardZkNodeName, String coreName) {
+ Map finalProps = new HashMap();
+ finalProps.put(ZkStateReader.BASE_URL_PROP, shardUrl);
+ finalProps.put(ZkStateReader.CORE_NAME_PROP, coreName);
+ finalProps.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+ finalProps.put(ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE);
+
+ publishState(cd, shardZkNodeName, coreName, finalProps);
+ }
+
+ public void publish(SolrCore core, String state) {
+ CoreDescriptor cd = core.getCoreDescriptor();
+ Map finalProps = new HashMap();
+ finalProps.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
+ finalProps.put(ZkStateReader.CORE_NAME_PROP, core.getName());
+ finalProps.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+ finalProps.put(ZkStateReader.STATE_PROP, state);
+ publishState(cd, getNodeName() + "_" + core.getName(),
+ core.getName(), finalProps);
+ }
+
+ void publishAsDown(String baseUrl,
+ final CoreDescriptor cd, String shardZkNodeName, String coreName) {
+ Map finalProps = new HashMap();
+ finalProps.put(ZkStateReader.BASE_URL_PROP, baseUrl);
+ finalProps.put(ZkStateReader.CORE_NAME_PROP, coreName);
+ finalProps.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+ finalProps.put(ZkStateReader.STATE_PROP, ZkStateReader.DOWN);
+
+ publishState(cd, shardZkNodeName, coreName, finalProps);
+ }
+
+ void publishAsRecoveryFailed(String baseUrl,
+ final CoreDescriptor cd, String shardZkNodeName, String coreName) {
+ Map finalProps = new HashMap();
+ finalProps.put(ZkStateReader.BASE_URL_PROP, baseUrl);
+ finalProps.put(ZkStateReader.CORE_NAME_PROP, coreName);
+ finalProps.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+ finalProps.put(ZkStateReader.STATE_PROP, ZkStateReader.RECOVERY_FAILED);
+ publishState(cd, shardZkNodeName, coreName, finalProps);
+ }
+
+
+ private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
+ final CloudState state, final String shardZkNodeName) {
+
+ final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
+
+ final String shardId = state.getShardId(shardZkNodeName);
+
+ if (shardId != null) {
+ cloudDesc.setShardId(shardId);
+ return false;
+ }
+ return true;
}
/**
@@ -513,16 +672,7 @@ public final class ZkController {
* @throws InterruptedException
*/
public void uploadToZK(File dir, String zkPath) throws IOException, KeeperException, InterruptedException {
- File[] files = dir.listFiles();
- for(File file : files) {
- if (!file.getName().startsWith(".")) {
- if (!file.isDirectory()) {
- zkClient.setData(zkPath + "/" + file.getName(), file);
- } else {
- uploadToZK(file, zkPath + "/" + file.getName());
- }
- }
- }
+ uploadToZK(zkClient, dir, zkPath);
}
/**
@@ -533,7 +683,7 @@ public final class ZkController {
* @throws InterruptedException
*/
public void uploadConfigDir(File dir, String configName) throws IOException, KeeperException, InterruptedException {
- uploadToZK(dir, ZkController.CONFIGS_ZKNODE + "/" + configName);
+ uploadToZK(zkClient, dir, ZkController.CONFIGS_ZKNODE + "/" + configName);
}
// convenience for testing
@@ -541,32 +691,6 @@ public final class ZkController {
zkClient.printLayoutToStdOut();
}
- private void setUpCollectionsNode() throws KeeperException, InterruptedException {
- try {
- if (!zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE)) {
- if (log.isInfoEnabled()) {
- log.info("creating zk collections node:" + ZkStateReader.COLLECTIONS_ZKNODE);
- }
- // makes collections zkNode if it doesn't exist
- zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE, CreateMode.PERSISTENT, null);
- }
- } catch (KeeperException e) {
- // its okay if another beats us creating the node
- if (e.code() != KeeperException.Code.NODEEXISTS) {
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- }
- } catch (InterruptedException e) {
- // Restore the interrupted status
- Thread.currentThread().interrupt();
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
- }
-
- }
-
public void createCollectionZkNode(CloudDescriptor cd) throws KeeperException, InterruptedException, IOException {
String collection = cd.getCollectionName();
@@ -574,12 +698,12 @@ public final class ZkController {
String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
try {
- if(!zkClient.exists(collectionPath)) {
+ if(!zkClient.exists(collectionPath, true)) {
log.info("Creating collection in ZooKeeper:" + collection);
SolrParams params = cd.getParams();
try {
- ZkNodeProps collectionProps = new ZkNodeProps();
+ Map collectionProps = new HashMap();
// TODO: if collection.configName isn't set, and there isn't already a conf in zk, just use that?
String defaultConfigName = System.getProperty(COLLECTION_PARAM_PREFIX+CONFIGNAME_PROP, "configuration1");
@@ -595,7 +719,7 @@ public final class ZkController {
// if the config name wasn't passed in, use the default
if (!collectionProps.containsKey(CONFIGNAME_PROP))
- collectionProps.put(CONFIGNAME_PROP, defaultConfigName);
+ getConfName(collection, collectionPath, collectionProps);
} else if(System.getProperty("bootstrap_confdir") != null) {
// if we are bootstrapping a collection, default the config for
@@ -614,32 +738,14 @@ public final class ZkController {
collectionProps.put(CONFIGNAME_PROP, defaultConfigName);
} else {
- // check for configName
- log.info("Looking for collection configName");
- int retry = 1;
- for (; retry < 6; retry++) {
- if (zkClient.exists(collectionPath)) {
- collectionProps = new ZkNodeProps();
- collectionProps.load(zkClient.getData(collectionPath, null, null));
- if (collectionProps.containsKey(CONFIGNAME_PROP)) {
- break;
- }
- }
- log.info("Could not find collection configName - pausing for 2 seconds and trying again - try: " + retry);
- Thread.sleep(2000);
- }
- if (retry == 6) {
- log.error("Could not find configName for collection " + collection);
- throw new ZooKeeperException(
- SolrException.ErrorCode.SERVER_ERROR,
- "Could not find configName for collection " + collection);
- }
+ getConfName(collection, collectionPath, collectionProps);
}
- zkClient.makePath(collectionPath, collectionProps.store(), CreateMode.PERSISTENT, null, true);
+ ZkNodeProps zkProps = new ZkNodeProps(collectionProps);
+ zkClient.makePath(collectionPath, ZkStateReader.toJSON(zkProps), CreateMode.PERSISTENT, null, true);
// ping that there is a new collection
- zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
+ zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null, true);
} catch (KeeperException e) {
// its okay if the node already exists
if (e.code() != KeeperException.Code.NODEEXISTS) {
@@ -658,9 +764,131 @@ public final class ZkController {
}
}
+
+
+ private void getConfName(String collection, String collectionPath,
+ Map collectionProps) throws KeeperException,
+ InterruptedException {
+ // check for configName
+ log.info("Looking for collection configName");
+ int retry = 1;
+ for (; retry < 6; retry++) {
+ if (zkClient.exists(collectionPath, true)) {
+ ZkNodeProps cProps = ZkNodeProps.load(zkClient.getData(collectionPath, null, null, true));
+ if (cProps.containsKey(CONFIGNAME_PROP)) {
+ break;
+ }
+ }
+ // if there is only one conf, use that
+ List configNames = zkClient.getChildren(CONFIGS_ZKNODE, null, true);
+ if (configNames.size() == 1) {
+ // no config set named, but there is only 1 - use it
+ log.info("Only one config set found in zk - using it:" + configNames.get(0));
+ collectionProps.put(CONFIGNAME_PROP, configNames.get(0));
+ break;
+ }
+ log.info("Could not find collection configName - pausing for 2 seconds and trying again - try: " + retry);
+ Thread.sleep(2000);
+ }
+ if (retry == 6) {
+ log.error("Could not find configName for collection " + collection);
+ throw new ZooKeeperException(
+ SolrException.ErrorCode.SERVER_ERROR,
+ "Could not find configName for collection " + collection);
+ }
+ }
public ZkStateReader getZkStateReader() {
return zkStateReader;
}
+
+ private void publishState(CoreDescriptor cd, String shardZkNodeName, String coreName,
+ Map props) {
+ CloudDescriptor cloudDesc = cd.getCloudDescriptor();
+
+ if (cloudDesc.getShardId() == null && needsToBeAssignedShardId(cd, zkStateReader.getCloudState(), shardZkNodeName)) {
+ // publish with no shard id so we are assigned one, and then look for it
+ doPublish(shardZkNodeName, coreName, props, cloudDesc);
+ String shardId;
+ try {
+ shardId = doGetShardIdProcess(coreName, cloudDesc);
+ } catch (InterruptedException e) {
+ throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted");
+ }
+ cloudDesc.setShardId(shardId);
+ }
+
+
+ if (!props.containsKey(ZkStateReader.SHARD_ID_PROP) && cloudDesc.getShardId() != null) {
+ props.put(ZkStateReader.SHARD_ID_PROP, cloudDesc.getShardId());
+ }
+
+ doPublish(shardZkNodeName, coreName, props, cloudDesc);
+ }
+
+
+ private void doPublish(String shardZkNodeName, String coreName,
+ Map props, CloudDescriptor cloudDesc) {
+
+ CoreState coreState = new CoreState(coreName,
+ cloudDesc.getCollectionName(), props);
+ coreStates.put(shardZkNodeName, coreState);
+ final String nodePath = "/node_states/" + getNodeName();
+
+ try {
+ zkClient.setData(nodePath, ZkStateReader.toJSON(coreStates.values()),
+ true);
+
+ } catch (KeeperException e) {
+ throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
+ "could not publish node state", e);
+ } catch (InterruptedException e) {
+ // Restore the interrupted status
+ Thread.currentThread().interrupt();
+ throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
+ "could not publish node state", e);
+ }
+ }
+
+ private String doGetShardIdProcess(String coreName, CloudDescriptor descriptor)
+ throws InterruptedException {
+ final String shardZkNodeName = getNodeName() + "_" + coreName;
+ int retryCount = 120;
+ while (retryCount-- > 0) {
+ final String shardId = zkStateReader.getCloudState().getShardId(
+ shardZkNodeName);
+ if (shardId != null) {
+ return shardId;
+ }
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ throw new SolrException(ErrorCode.SERVER_ERROR,
+ "Could not get shard_id for core: " + coreName);
+ }
+
+ public static void uploadToZK(SolrZkClient zkClient, File dir, String zkPath) throws IOException, KeeperException, InterruptedException {
+ File[] files = dir.listFiles();
+ if (files == null) {
+ throw new IllegalArgumentException("Illegal directory: " + dir);
+ }
+ for(File file : files) {
+ if (!file.getName().startsWith(".")) {
+ if (!file.isDirectory()) {
+ zkClient.makePath(zkPath + "/" + file.getName(), file, false, true);
+ } else {
+ uploadToZK(zkClient, file, zkPath + "/" + file.getName());
+ }
+ }
+ }
+ }
+
+ public static void uploadConfigDir(SolrZkClient zkClient, File dir, String configName) throws IOException, KeeperException, InterruptedException {
+ uploadToZK(zkClient, dir, ZkController.CONFIGS_ZKNODE + "/" + configName);
+ }
+
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
index 8a6f5957d52..46739f9d9d9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -74,7 +74,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
String file = collectionZkPath + "/" + resource;
try {
if (zkController.pathExists(file)) {
- byte[] bytes = zkController.getZkClient().getData(collectionZkPath + "/" + resource, null, null);
+ byte[] bytes = zkController.getZkClient().getData(collectionZkPath + "/" + resource, null, null, true);
return new ByteArrayInputStream(bytes);
}
} catch (Exception e) {
@@ -105,7 +105,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
public String[] listConfigDir() {
List list;
try {
- list = zkController.getZkClient().getChildren(collectionZkPath, null);
+ list = zkController.getZkClient().getChildren(collectionZkPath, null, true);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
@@ -119,5 +119,9 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
}
return list.toArray(new String[0]);
}
+
+ public String getCollectionZkPath() {
+ return collectionZkPath;
+ }
}
diff --git a/solr/core/src/java/org/apache/solr/core/Config.java b/solr/core/src/java/org/apache/solr/core/Config.java
index b5e8c3154fe..76ad9d52300 100644
--- a/solr/core/src/java/org/apache/solr/core/Config.java
+++ b/solr/core/src/java/org/apache/solr/core/Config.java
@@ -242,7 +242,10 @@ public class Config {
public String get(String path, String def) {
String val = getVal(path, false);
- return val!=null ? val : def;
+ if (val == null || val.length() == 0) {
+ return def;
+ }
+ return val;
}
public int getInt(String path) {
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 5ad878d568d..4fab1ac9c9b 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -23,6 +23,8 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeoutException;
import java.text.SimpleDateFormat;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
+import org.apache.solr.handler.component.ShardHandlerFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -32,10 +34,12 @@ import javax.xml.xpath.XPath;
import javax.xml.xpath.XPathExpressionException;
import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.cloud.CurrentCoreDescriptorProvider;
import org.apache.solr.cloud.SolrZkServer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.ZkSolrResourceLoader;
import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.DOMUtil;
@@ -85,6 +89,7 @@ public class CoreContainer
private SolrXMLSerializer solrXMLSerializer = new SolrXMLSerializer();
private ZkController zkController;
private SolrZkServer zkServer;
+ private ShardHandlerFactory shardHandlerFactory;
private String zkHost;
@@ -151,7 +156,7 @@ public class CoreContainer
zookeeperHost = zkServer.getClientString();
}
- int zkClientConnectTimeout = 5000;
+ int zkClientConnectTimeout = 15000;
if (zookeeperHost != null) {
// we are ZooKeeper enabled
@@ -163,7 +168,17 @@ public class CoreContainer
} else {
log.info("Zookeeper client=" + zookeeperHost);
}
- zkController = new ZkController(zookeeperHost, zkClientTimeout, zkClientConnectTimeout, host, hostPort, hostContext);
+ zkController = new ZkController(this, zookeeperHost, zkClientTimeout, zkClientConnectTimeout, host, hostPort, hostContext, new CurrentCoreDescriptorProvider() {
+
+ @Override
+ public List getCurrentDescriptors() {
+ List descriptors = new ArrayList(getCoreNames().size());
+ for (SolrCore core : getCores()) {
+ descriptors.add(core.getCoreDescriptor());
+ }
+ return descriptors;
+ }
+ });
String confDir = System.getProperty("bootstrap_confdir");
if(confDir != null) {
@@ -203,7 +218,7 @@ public class CoreContainer
// Helper class to initialize the CoreContainer
public static class Initializer {
- protected String solrConfigFilename = null;
+ protected String containerConfigFilename = null; // normally "solr.xml"
protected String dataDir = null; // override datadir for single core mode
// core container instantiation
@@ -211,9 +226,8 @@ public class CoreContainer
ParserConfigurationException, SAXException {
CoreContainer cores = null;
String solrHome = SolrResourceLoader.locateSolrHome();
- // TODO : fix broken logic confusing solr.xml with solrconfig.xml
- File fconf = new File(solrHome, solrConfigFilename == null ? "solr.xml"
- : solrConfigFilename);
+ File fconf = new File(solrHome, containerConfigFilename == null ? "solr.xml"
+ : containerConfigFilename);
log.info("looking for solr.xml: " + fconf.getAbsolutePath());
cores = new CoreContainer();
@@ -225,10 +239,7 @@ public class CoreContainer
cores.configFile = fconf;
}
- solrConfigFilename = cores.getConfigFile().getName();
- if (cores.cores.isEmpty()){
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No cores were created, please check the logs for errors");
- }
+ containerConfigFilename = cores.getConfigFile().getName();
return cores;
}
@@ -300,10 +311,7 @@ public class CoreContainer
shareSchema = cfg.getBool("solr/cores/@shareSchema", false);
zkClientTimeout = cfg.getInt("solr/cores/@zkClientTimeout", 10000);
- hostPort = System.getProperty("hostPort");
- if (hostPort == null) {
- hostPort = cfg.get("solr/cores/@hostPort", "8983");
- }
+ hostPort = cfg.get("solr/cores/@hostPort", "8983");
hostContext = cfg.get("solr/cores/@hostContext", "solr");
host = cfg.get("solr/cores/@host", null);
@@ -338,7 +346,7 @@ public class CoreContainer
}
NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core", XPathConstants.NODESET);
- boolean defaultCoreFound = false;
+
for (int i=0; iemptyList());
+
+ HttpShardHandlerFactory fac = new HttpShardHandlerFactory();
+ fac.init(info);
+ shardHandlerFactory = fac;
+ }
+ return shardHandlerFactory;
+ }
+ }
private SolrConfig getSolrConfigFromZk(String zkConfigName, String solrConfigFileName,
SolrResourceLoader resourceLoader) throws IOException,
@@ -976,7 +1012,7 @@ public class CoreContainer
private static final String DEF_SOLR_XML ="\n" +
"\n" +
" \n" +
- " \n" +
+ " \n" +
" \n" +
"";
}
diff --git a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
index 4a316cd211b..1fdeef9b05d 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
@@ -43,11 +43,10 @@ public class CoreDescriptor {
this.coreContainer = coreContainer;
this.name = name;
- if(coreContainer.getZkController() != null) {
+ if(coreContainer != null && coreContainer.getZkController() != null) {
this.cloudDesc = new CloudDescriptor();
// cloud collection defaults to core name
cloudDesc.setCollectionName(name.isEmpty() ? coreContainer.getDefaultCoreName() : name);
- this.cloudDesc.setShardId(coreContainer.getZkController().getNodeName() + "_" + name);
}
if (name == null) {
diff --git a/solr/core/src/java/org/apache/solr/core/RequestHandlers.java b/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
index 587fc51e603..daff33c57b8 100644
--- a/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
+++ b/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
@@ -26,11 +26,8 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.CommonParams.EchoParamStyle;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.handler.component.SearchHandler;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.response.SolrQueryResponse;
@@ -41,7 +38,7 @@ import org.slf4j.LoggerFactory;
/**
*/
-final class RequestHandlers {
+public final class RequestHandlers {
public static Logger log = LoggerFactory.getLogger(RequestHandlers.class);
public static final String DEFAULT_HANDLER_NAME="standard";
@@ -208,7 +205,7 @@ final class RequestHandlers {
*
* @since solr 1.2
*/
- private static final class LazyRequestHandlerWrapper implements SolrRequestHandler, SolrInfoMBean
+ public static final class LazyRequestHandlerWrapper implements SolrRequestHandler, SolrInfoMBean
{
private final SolrCore core;
private String _className;
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index cce442d9382..3c4c6f53855 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -42,10 +42,7 @@ import org.apache.solr.search.ValueSourceParser;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.SolrIndexWriter;
import org.apache.solr.update.UpdateHandler;
-import org.apache.solr.update.processor.LogUpdateProcessorFactory;
-import org.apache.solr.update.processor.RunUpdateProcessorFactory;
-import org.apache.solr.update.processor.UpdateRequestProcessorChain;
-import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
+import org.apache.solr.update.processor.*;
import org.apache.solr.util.RefCounted;
import org.apache.solr.util.plugin.NamedListInitializedPlugin;
import org.apache.solr.util.plugin.SolrCoreAware;
@@ -63,6 +60,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.URL;
import java.lang.reflect.Constructor;
+import java.util.concurrent.locks.ReentrantLock;
/**
@@ -77,6 +75,8 @@ public final class SolrCore implements SolrInfoMBean {
private String logid; // used to show what name is set
private final CoreDescriptor coreDescriptor;
+ private boolean isReloaded = false;
+
private final SolrConfig solrConfig;
private final SolrResourceLoader resourceLoader;
private final IndexSchema schema;
@@ -475,20 +475,6 @@ public final class SolrCore implements SolrInfoMBean {
return createInstance(className, QueryResponseWriter.class, "Query Response Writer");
}
- /**
- *
- * @param dataDir
- * @param schema
- * @throws SAXException
- * @throws IOException
- * @throws ParserConfigurationException
- *
- * @since solr 1.0
- */
- public SolrCore(String dataDir, IndexSchema schema) throws ParserConfigurationException, IOException, SAXException {
- this(null, dataDir, new SolrConfig(), schema, null);
- }
-
/**
* Creates a new core and register it in the list of cores.
* If a core with the same name already exists, it will be stopped and replaced by this one.
@@ -558,7 +544,8 @@ public final class SolrCore implements SolrInfoMBean {
if (updateHandler == null) {
initDirectoryFactory();
} else {
- directoryFactory = updateHandler.getIndexWriterProvider().getDirectoryFactory();
+ directoryFactory = updateHandler.getSolrCoreState().getDirectoryFactory();
+ this.isReloaded = true;
}
initIndex();
@@ -658,6 +645,7 @@ public final class SolrCore implements SolrInfoMBean {
// construct the default chain
UpdateRequestProcessorFactory[] factories = new UpdateRequestProcessorFactory[]{
new LogUpdateProcessorFactory(),
+ new DistributedUpdateProcessorFactory(),
new RunUpdateProcessorFactory()
};
def = new UpdateRequestProcessorChain(factories, this);
@@ -762,7 +750,7 @@ public final class SolrCore implements SolrInfoMBean {
try {
updateHandler.close();
- } catch (Exception e) {
+ } catch (Throwable e) {
SolrException.log(log,e);
}
@@ -960,15 +948,21 @@ public final class SolrCore implements SolrInfoMBean {
// This reference is protected by searcherLock.
private RefCounted _searcher;
- // All of the open searchers. Don't access this directly.
+ // All of the normal open searchers. Don't access this directly.
// protected by synchronizing on searcherLock.
private final LinkedList> _searchers = new LinkedList>();
+ private final LinkedList> _realtimeSearchers = new LinkedList>();
final ExecutorService searcherExecutor = Executors.newSingleThreadExecutor();
private int onDeckSearchers; // number of searchers preparing
+ // Lock ordering: one can acquire the openSearcherLock and then the searcherLock, but not vice-versa.
private Object searcherLock = new Object(); // the sync object for the searcher
+ private ReentrantLock openSearcherLock = new ReentrantLock(true); // used to serialize opens/reopens for absolute ordering
private final int maxWarmingSearchers; // max number of on-deck searchers allowed
+ private RefCounted realtimeSearcher;
+
+
/**
* Return a registered {@link RefCounted}<{@link SolrIndexSearcher}> with
* the reference count incremented. It must be decremented when no longer needed.
@@ -987,29 +981,162 @@ public final class SolrCore implements SolrInfoMBean {
}
/**
- * Return the newest {@link RefCounted}<{@link SolrIndexSearcher}> with
+ * Return the newest normal {@link RefCounted}<{@link SolrIndexSearcher}> with
* the reference count incremented. It must be decremented when no longer needed.
* If no searcher is currently open, then if openNew==true a new searcher will be opened,
* or null is returned if openNew==false.
*/
public RefCounted getNewestSearcher(boolean openNew) {
synchronized (searcherLock) {
- if (_searchers.isEmpty()) {
- if (!openNew) return null;
- // Not currently implemented since simply calling getSearcher during inform()
- // can result in a deadlock. Right now, solr always opens a searcher first
- // before calling inform() anyway, so this should never happen.
- throw new UnsupportedOperationException();
+ if (!_searchers.isEmpty()) {
+ RefCounted newest = _searchers.getLast();
+ newest.incref();
+ return newest;
}
- RefCounted newest = _searchers.getLast();
- newest.incref();
- return newest;
+ }
+
+ return openNew ? getRealtimeSearcher() : null;
+ }
+
+
+ /** Gets the latest real-time searcher w/o forcing open a new searcher if one already exists.
+ * The reference count will be incremented.
+ */
+ public RefCounted getRealtimeSearcher() {
+ synchronized (searcherLock) {
+ if (realtimeSearcher != null) {
+ realtimeSearcher.incref();
+ return realtimeSearcher;
+ }
+ }
+
+ // use the searcher lock to prevent multiple people from trying to open at once
+ openSearcherLock.lock();
+ try {
+
+ // try again
+ synchronized (searcherLock) {
+ if (realtimeSearcher != null) {
+ realtimeSearcher.incref();
+ return realtimeSearcher;
+ }
+ }
+
+ // force a new searcher open
+ return openNewSearcher(true, true);
+ } finally {
+ openSearcherLock.unlock();
}
}
+
public RefCounted getSearcher(boolean forceNew, boolean returnSearcher, final Future[] waitSearcher) throws IOException {
return getSearcher(forceNew, returnSearcher, waitSearcher, false);
}
+
+
+ /** Opens a new searcher and returns a RefCounted with it's reference incremented.
+ *
+ * "realtime" means that we need to open quickly for a realtime view of the index, hence don't do any
+ * autowarming and add to the _realtimeSearchers queue rather than the _searchers queue (so it won't
+ * be used for autowarming by a future normal searcher). A "realtime" searcher will currently never
+ * become "registered" (since it currently lacks caching).
+ *
+ * realtimeSearcher is updated to the latest opened searcher, regardless of the value of "realtime".
+ *
+ * This method aquires openSearcherLock - do not call with searckLock held!
+ */
+ public RefCounted openNewSearcher(boolean updateHandlerReopens, boolean realtime) {
+ SolrIndexSearcher tmp;
+ RefCounted newestSearcher = null;
+ boolean nrt = solrConfig.reopenReaders && updateHandlerReopens;
+
+ openSearcherLock.lock();
+ try {
+ String newIndexDir = null;
+ File indexDirFile = null;
+ File newIndexDirFile = null;
+
+ // if it's not a normal near-realtime update, check that paths haven't changed.
+ if (!nrt) {
+ indexDirFile = new File(getIndexDir()).getCanonicalFile();
+ newIndexDir = getNewIndexDir();
+ newIndexDirFile = new File(newIndexDir).getCanonicalFile();
+ }
+
+ synchronized (searcherLock) {
+ newestSearcher = realtimeSearcher;
+ if (newestSearcher != null) {
+ newestSearcher.incref(); // the matching decref is in the finally block
+ }
+ }
+
+ if (newestSearcher != null && solrConfig.reopenReaders
+ && (nrt || indexDirFile.equals(newIndexDirFile))) {
+
+ IndexReader newReader;
+ IndexReader currentReader = newestSearcher.get().getIndexReader();
+
+ if (updateHandlerReopens) {
+ // SolrCore.verbose("start reopen from",previousSearcher,"writer=",writer);
+ IndexWriter writer = getUpdateHandler().getSolrCoreState().getIndexWriter(this);
+ newReader = IndexReader.openIfChanged(currentReader, writer, true);
+
+ } else {
+ // verbose("start reopen without writer, reader=", currentReader);
+ newReader = IndexReader.openIfChanged(currentReader);
+ // verbose("reopen result", newReader);
+ }
+
+ if (newReader == null) {
+ // if this is a request for a realtime searcher, just return the same searcher if there haven't been any changes.
+ if (realtime) {
+ newestSearcher.incref();
+ return newestSearcher;
+ }
+
+ currentReader.incRef();
+ newReader = currentReader;
+ }
+
+ // for now, turn off caches if this is for a realtime reader (caches take a little while to instantiate)
+ tmp = new SolrIndexSearcher(this, schema, (realtime ? "realtime":"main"), newReader, true, !realtime, true, directoryFactory);
+
+ } else {
+ // verbose("non-reopen START:");
+ tmp = new SolrIndexSearcher(this, newIndexDir, schema, getSolrConfig().mainIndexConfig, "main", true, directoryFactory);
+ // verbose("non-reopen DONE: searcher=",tmp);
+ }
+
+ List> searcherList = realtime ? _realtimeSearchers : _searchers;
+ RefCounted newSearcher = newHolder(tmp, searcherList); // refcount now at 1
+
+ // Increment reference again for "realtimeSearcher" variable. It should be at 2 after.
+ // When it's decremented by both the caller of this method, and by realtimeSearcher being replaced,
+ // it will be closed.
+ newSearcher.incref();
+
+ synchronized (searcherLock) {
+ if (realtimeSearcher != null) {
+ realtimeSearcher.decref();
+ }
+ realtimeSearcher = newSearcher;
+ searcherList.add(realtimeSearcher);
+ }
+
+ return newSearcher;
+
+ } catch (Exception e) {
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error opening new searcher", e);
+ }
+ finally {
+ openSearcherLock.unlock();
+ if (newestSearcher != null) {
+ newestSearcher.decref();
+ }
+ }
+
+ }
/**
* Get a {@link SolrIndexSearcher} or start the process of creating a new one.
@@ -1105,80 +1232,28 @@ public final class SolrCore implements SolrInfoMBean {
}
}
- // open the index synchronously
- // if this fails, we need to decrement onDeckSearchers again.
- SolrIndexSearcher tmp;
- RefCounted newestSearcher = null;
-
- try {
- newestSearcher = getNewestSearcher(false);
- String newIndexDir = getNewIndexDir();
- File indexDirFile = new File(getIndexDir()).getCanonicalFile();
- File newIndexDirFile = new File(newIndexDir).getCanonicalFile();
-
- if (newestSearcher != null && solrConfig.reopenReaders
- && indexDirFile.equals(newIndexDirFile)) {
-
- if (updateHandlerReopens) {
-
- tmp = getUpdateHandler().reopenSearcher(newestSearcher.get());
- } else {
-
- IndexReader currentReader = newestSearcher.get().getIndexReader();
- IndexReader newReader;
-
- // verbose("start reopen without writer, reader=", currentReader);
- newReader = IndexReader.openIfChanged(currentReader);
- // verbose("reopen result", newReader);
-
-
- if (newReader == null) {
- currentReader.incRef();
- newReader = currentReader;
- }
-
- tmp = new SolrIndexSearcher(this, schema, "main", newReader, true, true, true, directoryFactory);
- }
-
-
- } else {
- // verbose("non-reopen START:");
- tmp = new SolrIndexSearcher(this, newIndexDir, schema, getSolrConfig().mainIndexConfig, "main", true, directoryFactory);
- // verbose("non-reopen DONE: searcher=",tmp);
- }
- } catch (Throwable th) {
- synchronized(searcherLock) {
- onDeckSearchers--;
- // notify another waiter to continue... it may succeed
- // and wake any others.
- searcherLock.notify();
- }
- // need to close the searcher here??? we shouldn't have to.
- throw new RuntimeException(th);
- } finally {
- if (newestSearcher != null) {
- newestSearcher.decref();
- }
- }
-
- final SolrIndexSearcher newSearcher=tmp;
-
- RefCounted currSearcherHolder=null;
- final RefCounted newSearchHolder=newHolder(newSearcher);
-
- if (returnSearcher) newSearchHolder.incref();
-
// a signal to decrement onDeckSearchers if something goes wrong.
- final boolean[] decrementOnDeckCount=new boolean[1];
- decrementOnDeckCount[0]=true;
+ final boolean[] decrementOnDeckCount=new boolean[]{true};
+ RefCounted currSearcherHolder = null; // searcher we are autowarming from
+ RefCounted searchHolder = null;
+ boolean success = false;
+ openSearcherLock.lock();
try {
+ searchHolder = openNewSearcher(updateHandlerReopens, false);
+ // the searchHolder will be incremented once already (and it will eventually be assigned to _searcher when registered)
+ // increment it again if we are going to return it to the caller.
+ if (returnSearcher) {
+ searchHolder.incref();
+ }
+
+
+ final RefCounted newSearchHolder = searchHolder;
+ final SolrIndexSearcher newSearcher = newSearchHolder.get();
+
boolean alreadyRegistered = false;
synchronized (searcherLock) {
- _searchers.add(newSearchHolder);
- // verbose("added searcher ",newSearchHolder.get()," to _searchers");
-
if (_searcher == null) {
// if there isn't a current searcher then we may
// want to register this one before warming is complete instead of waiting.
@@ -1197,174 +1272,133 @@ public final class SolrCore implements SolrInfoMBean {
final SolrIndexSearcher currSearcher = currSearcherHolder==null ? null : currSearcherHolder.get();
- //
- // Note! if we registered the new searcher (but didn't increment it's
- // reference count because returnSearcher==false, it's possible for
- // someone else to register another searcher, and thus cause newSearcher
- // to close while we are warming.
- //
- // Should we protect against that by incrementing the reference count?
- // Maybe we should just let it fail? After all, if returnSearcher==false
- // and newSearcher has been de-registered, what's the point of continuing?
- //
-
Future future=null;
// warm the new searcher based on the current searcher.
// should this go before the other event handlers or after?
if (currSearcher != null) {
- try {
- future = searcherExecutor.submit(
- new Callable() {
- public Object call() throws Exception {
- try {
- newSearcher.warm(currSearcher);
- } catch (Throwable e) {
- SolrException.log(log, null, e);
- }
- return null;
- }
- }
- );
- } catch(Exception e) {
- // if submit fails, newSearchHolder does not get decref'd
- if (newSearchHolder != null) {
- newSearchHolder.decref();
- if (returnSearcher) {
- newSearchHolder.decref();
+ future = searcherExecutor.submit(
+ new Callable() {
+ public Object call() throws Exception {
+ try {
+ newSearcher.warm(currSearcher);
+ } catch (Throwable e) {
+ SolrException.log(log,e);
+ }
+ return null;
+ }
}
- }
- throw e;
- }
+ );
}
-
+
if (currSearcher==null && firstSearcherListeners.size() > 0) {
- try {
- future = searcherExecutor.submit(
- new Callable() {
- public Object call() throws Exception {
- try {
- for (SolrEventListener listener : firstSearcherListeners) {
- listener.newSearcher(newSearcher,null);
- }
- } catch (Throwable e) {
- SolrException.log(log, null, e);
- }
- return null;
- }
+ future = searcherExecutor.submit(
+ new Callable() {
+ public Object call() throws Exception {
+ try {
+ for (SolrEventListener listener : firstSearcherListeners) {
+ listener.newSearcher(newSearcher,null);
}
- );
- } catch(Exception e) {
- // if submit fails, newSearchHolder does not get decref'd
- if (newSearchHolder != null) {
- newSearchHolder.decref();
- if (returnSearcher) {
- newSearchHolder.decref();
+ } catch (Throwable e) {
+ SolrException.log(log,null,e);
+ }
+ return null;
+ }
}
- }
- throw e;
- }
+ );
}
if (currSearcher!=null && newSearcherListeners.size() > 0) {
- try {
- future = searcherExecutor.submit(
- new Callable() {
- public Object call() throws Exception {
- try {
- for (SolrEventListener listener : newSearcherListeners) {
- listener.newSearcher(newSearcher, currSearcher);
- }
- } catch (Throwable e) {
- SolrException.log(log, null, e);
- }
- return null;
- }
+ future = searcherExecutor.submit(
+ new Callable() {
+ public Object call() throws Exception {
+ try {
+ for (SolrEventListener listener : newSearcherListeners) {
+ listener.newSearcher(newSearcher, currSearcher);
}
- );
- } catch(Exception e) {
- // if submit fails, newSearchHolder does not get decref'd
- if (newSearchHolder != null) {
- newSearchHolder.decref();
- if (returnSearcher) {
- newSearchHolder.decref();
- }
- }
- throw e;
- }
+ } catch (Throwable e) {
+ SolrException.log(log,null,e);
+ }
+ return null;
+ }
+ }
+ );
}
// WARNING: this code assumes a single threaded executor (that all tasks
// queued will finish first).
final RefCounted currSearcherHolderF = currSearcherHolder;
if (!alreadyRegistered) {
- try {
- future = searcherExecutor.submit(
- new Callable() {
- public Object call() throws Exception {
- try {
- // signal that we no longer need to decrement
- // the count *before* registering the searcher since
- // registerSearcher will decrement even if it errors.
- decrementOnDeckCount[0]=false;
- registerSearcher(newSearchHolder);
- } catch (Throwable e) {
- SolrException.log(log, null, e);
- } finally {
- // we are all done with the old searcher we used
- // for warming...
- if (currSearcherHolderF!=null) currSearcherHolderF.decref();
- }
- return null;
- }
- }
- );
- } catch(Exception e) {
- // if submit fails, newSearchHolder does not get decref'd
- if (newSearchHolder != null) {
- newSearchHolder.decref();
- if (returnSearcher) {
- newSearchHolder.decref();
+ future = searcherExecutor.submit(
+ new Callable() {
+ public Object call() throws Exception {
+ try {
+ // registerSearcher will decrement onDeckSearchers and
+ // do a notify, even if it fails.
+ registerSearcher(newSearchHolder);
+ } catch (Throwable e) {
+ SolrException.log(log, e);
+ } finally {
+ // we are all done with the old searcher we used
+ // for warming...
+ if (currSearcherHolderF!=null) currSearcherHolderF.decref();
+ }
+ return null;
+ }
}
- }
- throw e;
- }
+ );
}
if (waitSearcher != null) {
waitSearcher[0] = future;
}
+ success = true;
+
// Return the searcher as the warming tasks run in parallel
// callers may wait on the waitSearcher future returned.
return returnSearcher ? newSearchHolder : null;
} catch (Exception e) {
- SolrException.log(log, null, e);
- if (currSearcherHolder != null) currSearcherHolder.decref();
+ if (e instanceof SolrException) throw (SolrException)e;
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ } finally {
- synchronized (searcherLock) {
- if (decrementOnDeckCount[0]) {
+ if (!success) {
+ synchronized (searcherLock) {
onDeckSearchers--;
+
+ if (onDeckSearchers < 0) {
+ // sanity check... should never happen
+ log.error(logid+"ERROR!!! onDeckSearchers after decrement=" + onDeckSearchers);
+ onDeckSearchers=0; // try and recover
+ }
+ // if we failed, we need to wake up at least one waiter to continue the process
+ searcherLock.notify();
}
- if (onDeckSearchers < 0) {
- // sanity check... should never happen
- log.error(logid+"ERROR!!! onDeckSearchers after decrement=" + onDeckSearchers);
- onDeckSearchers=0; // try and recover
+
+ if (currSearcherHolder != null) {
+ currSearcherHolder.decref();
+ }
+
+ if (searchHolder != null) {
+ searchHolder.decref(); // decrement 1 for _searcher (searchHolder will never become _searcher now)
+ if (returnSearcher) {
+ searchHolder.decref(); // decrement 1 because we won't be returning the searcher to the user
+ }
}
- // if we failed, we need to wake up at least one waiter to continue the process
- searcherLock.notify();
}
- // since the indexreader was already opened, assume we can continue on
- // even though we got an exception.
- return returnSearcher ? newSearchHolder : null;
+ // we want to do this after we decrement onDeckSearchers so another thread
+ // doesn't increment first and throw a false warning.
+ openSearcherLock.unlock();
+
}
}
- private RefCounted newHolder(SolrIndexSearcher newSearcher) {
+ private RefCounted newHolder(SolrIndexSearcher newSearcher, final List> searcherList) {
RefCounted holder = new RefCounted(newSearcher) {
@Override
public void close() {
@@ -1376,11 +1410,13 @@ public final class SolrCore implements SolrInfoMBean {
// This relies on the RefCounted class allowing close() to be called every
// time the counter hits zero.
if (refcount.get() > 0) return;
- _searchers.remove(this);
+ searcherList.remove(this);
}
resource.close();
- } catch (IOException e) {
- log.error("Error closing searcher:" + SolrException.toStr(e));
+ } catch (Throwable e) {
+ // do not allow decref() operations to fail since they are typically called in finally blocks
+ // and throwing another exception would be very unexpected.
+ SolrException.log(log, "Error closing searcher:", e);
}
}
};
@@ -1388,6 +1424,9 @@ public final class SolrCore implements SolrInfoMBean {
return holder;
}
+ public boolean isReloaded() {
+ return isReloaded;
+ }
// Take control of newSearcherHolder (which should have a reference count of at
// least 1 already. If the caller wishes to use the newSearcherHolder directly
@@ -1423,6 +1462,7 @@ public final class SolrCore implements SolrInfoMBean {
log.info(logid+"Registered new searcher " + newSearcher);
} catch (Throwable e) {
+ // an exception in register() shouldn't be fatal.
log(e);
} finally {
// wake up anyone waiting for a searcher
@@ -1438,9 +1478,13 @@ public final class SolrCore implements SolrInfoMBean {
public void closeSearcher() {
log.info(logid+"Closing main searcher on request.");
synchronized (searcherLock) {
+ if (realtimeSearcher != null) {
+ realtimeSearcher.decref();
+ realtimeSearcher = null;
+ }
if (_searcher != null) {
_searcher.decref(); // dec refcount for this._searcher
- _searcher=null; // isClosed() does check this
+ _searcher = null; // isClosed() does check this
infoRegistry.remove("currentSearcher");
}
}
@@ -1470,7 +1514,7 @@ public final class SolrCore implements SolrInfoMBean {
handler.handleRequest(req,rsp);
setResponseHeaderValues(handler,req,rsp);
-
+
if (log.isInfoEnabled()) {
StringBuilder sb = new StringBuilder(logid);
for (int i=0; i();
+ }
+
if (enableMaster) {
includeConfFiles = (String) master.get(CONF_FILES);
if (includeConfFiles != null && includeConfFiles.trim().length() > 0) {
@@ -796,6 +809,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
replicateOnCommit = replicateAfter.contains("commit");
replicateOnOptimize = !replicateOnCommit && replicateAfter.contains("optimize");
+ if (!replicateOnCommit && ! replicateOnOptimize) {
+ replicateOnCommit = true;
+ }
+
// if we only want to replicate on optimize, we need the deletion policy to
// save the last optimized commit point.
if (replicateOnOptimize) {
diff --git a/solr/core/src/java/org/apache/solr/handler/SnapPuller.java b/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
index 8081bc48b90..c6c25325b7f 100644
--- a/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
+++ b/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
@@ -33,6 +33,7 @@ import static org.apache.solr.handler.ReplicationHandler.*;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.util.RefCounted;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -158,7 +159,7 @@ public class SnapPuller {
}
try {
executorStartTime = System.currentTimeMillis();
- replicationHandler.doFetch(null);
+ replicationHandler.doFetch(null, false);
} catch (Exception e) {
LOG.error("Exception in fetching index", e);
}
@@ -243,7 +244,8 @@ public class SnapPuller {
@SuppressWarnings("unchecked")
boolean successfulInstall = false;
- boolean fetchLatestIndex(SolrCore core) throws IOException {
+ boolean fetchLatestIndex(SolrCore core, boolean force) throws IOException, InterruptedException {
+ successfulInstall = false;
replicationStartTime = System.currentTimeMillis();
try {
//get the current 'replicateable' index version in the master
@@ -256,23 +258,41 @@ public class SnapPuller {
}
long latestVersion = (Long) response.get(CMD_INDEX_VERSION);
long latestGeneration = (Long) response.get(GENERATION);
- if (latestVersion == 0L) {
- //there is nothing to be replicated
- return false;
- }
+
IndexCommit commit;
RefCounted searcherRefCounted = null;
try {
searcherRefCounted = core.getNewestSearcher(false);
+ if (searcherRefCounted == null) {
+ SolrException.log(LOG, "No open searcher found - fetch aborted");
+ return false;
+ }
commit = searcherRefCounted.get().getIndexReader().getIndexCommit();
} finally {
if (searcherRefCounted != null)
searcherRefCounted.decref();
}
+
+ if (latestVersion == 0L) {
+ if (force && commit.getVersion() != 0) {
+ // since we won't get the files for an empty index,
+ // we just clear ours and commit
+ core.getUpdateHandler().getSolrCoreState().getIndexWriter(core).deleteAll();
+ SolrQueryRequest req = new LocalSolrQueryRequest(core,
+ new ModifiableSolrParams());
+ core.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
+ }
+
+ //there is nothing to be replicated
+ successfulInstall = true;
+ return true;
+ }
+
if (commit.getVersion() == latestVersion && commit.getGeneration() == latestGeneration) {
- //master and slave are alsready in sync just return
+ //master and slave are already in sync just return
LOG.info("Slave in sync with master.");
- return false;
+ successfulInstall = true;
+ return true;
}
LOG.info("Master's version: " + latestVersion + ", generation: " + latestGeneration);
LOG.info("Slave's version: " + commit.getVersion() + ", generation: " + commit.getGeneration());
@@ -289,7 +309,7 @@ public class SnapPuller {
filesDownloaded = Collections.synchronizedList(new ArrayList