SOLR-2358: merge in solrcloud branch (watch out hudson!)

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1235888 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2012-01-25 19:49:26 +00:00
parent 1e79c3b98e
commit 9fed484fb1
156 changed files with 16372 additions and 2560 deletions

View File

@ -100,7 +100,7 @@
<classpathentry kind="lib" path="modules/benchmark/lib/commons-digester-1.7.jar"/>
<classpathentry kind="lib" path="modules/benchmark/lib/commons-logging-1.0.4.jar"/>
<classpathentry kind="lib" path="modules/benchmark/lib/xercesImpl-2.9.1-patched-XERCESJ-1257.jar"/>
<classpathentry kind="lib" path="solr/lib/apache-solr-noggit-r1209632.jar"/>
<classpathentry kind="lib" path="solr/lib/apache-solr-noggit-r1211150.jar"/>
<classpathentry kind="lib" path="solr/lib/commons-csv-1.0-SNAPSHOT-r966014.jar"/>
<classpathentry kind="lib" path="solr/lib/commons-fileupload-1.2.1.jar"/>
<classpathentry kind="lib" path="solr/lib/commons-httpclient-3.1.jar"/>
@ -115,7 +115,7 @@
<classpathentry kind="lib" path="solr/lib/slf4j-api-1.6.1.jar"/>
<classpathentry kind="lib" path="solr/lib/slf4j-jdk14-1.6.1.jar"/>
<classpathentry kind="lib" path="solr/lib/wstx-asl-3.2.7.jar"/>
<classpathentry kind="lib" path="solr/lib/zookeeper-3.3.3.jar"/>
<classpathentry kind="lib" path="solr/lib/zookeeper-3.3.4.jar"/>
<classpathentry kind="lib" path="solr/example/lib/jetty-6.1.26-patched-JETTY-1340.jar"/>
<classpathentry kind="lib" path="solr/example/lib/jetty-util-6.1.26-patched-JETTY-1340.jar"/>
<classpathentry kind="lib" path="solr/example/lib/servlet-api-2.5-20081211.jar"/>

View File

@ -283,7 +283,7 @@
<dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<version>3.3.3</version>
<version>3.3.4</version>
</dependency>
<dependency>
<groupId>org.carrot2</groupId>
@ -670,7 +670,7 @@
<artifactId>solr-noggit</artifactId>
<version>${project.version}</version>
<packaging>jar</packaging>
<file>solr/lib/apache-solr-noggit-r1209632.jar</file>
<file>solr/lib/apache-solr-noggit-r1211150.jar</file>
</configuration>
</execution>
<execution>

View File

@ -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

View File

@ -482,7 +482,7 @@
<packageset dir="contrib/langid/src/java"/>
<packageset dir="contrib/uima/src/java"/>
<group title="Core" packages="org.apache.*" />
<group title="SolrJ" packages="org.apache.solr.common.*,org.apache.solr.client.solrj*" />
<group title="SolrJ" packages="org.apache.solr.common.*,org.apache.solr.client.solrj.*,org.apache.zookeeper.*" />
<group title="contrib: Clustering" packages="org.apache.solr.handler.clustering*" />
<group title="contrib: DataImportHandler" packages="org.apache.solr.handler.dataimport*" />
<group title="contrib: Solr Cell" packages="org.apache.solr.handler.extraction*" />

View File

@ -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 &

View File

@ -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 &

View File

@ -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 &

View File

@ -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 &

10
solr/cloud-dev/stop.sh Normal file
View File

@ -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

View File

@ -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);

View File

@ -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;
}

View File

@ -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";
@ -69,28 +69,35 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
SOLR_DOCS.add(solrDoc);
}
private static final String DIH_CONFIG_TAGS_INNER_ENTITY = "<dataConfig>\r\n"
private SolrInstance instance = null;
private JettySolrRunner jetty;
private static String getDihConfigTagsInnerEntity(int port) {
return "<dataConfig>\r\n"
+ " <dataSource type='MockDataSource' />\r\n"
+ " <document>\r\n"
+ " <entity name='db' query='select * from x'>\r\n"
+ " <field column='dbid_s' />\r\n"
+ " <field column='dbdesc_s' />\r\n"
+ " <entity name='se' processor='SolrEntityProcessor' query='id:${db.dbid_s}'\n"
+ " url='" + SOLR_SOURCE_URL + "' fields='id,desc'>\r\n"
+ " url='" + getSourceUrl(port) + "' fields='id,desc'>\r\n"
+ " <field column='id' />\r\n"
+ " <field column='desc' />\r\n" + " </entity>\r\n"
+ " </entity>\r\n" + " </document>\r\n" + "</dataConfig>\r\n";
}
private SolrInstance instance = null;
private JettySolrRunner jetty;
private static String generateDIHConfig(String options) {
private static String generateDIHConfig(String options, int port) {
return "<dataConfig>\r\n" + " <document>\r\n"
+ " <entity name='se' processor='SolrEntityProcessor'" + " url='"
+ SOLR_SOURCE_URL + "' " + options + " />\r\n" + " </document>\r\n"
+ getSourceUrl(port) + "' " + options + " />\r\n" + " </document>\r\n"
+ "</dataConfig>\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
@ -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<String,String> map = new HashMap<String,String>();
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<String,String> map = new HashMap<String,String>();
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;
}

View File

@ -27,6 +27,6 @@
jar.file="${common-solr.dir}/lib/commons-csv-1.0-SNAPSHOT-r966014.jar" />
<m2-deploy-with-pom-template pom.xml="${common-solr.dir}/lib/apache-solr-noggit-pom.xml.template"
jar.file="${common-solr.dir}/lib/apache-solr-noggit-r1209632.jar" />
jar.file="${common-solr.dir}/lib/apache-solr-noggit-r1211150.jar" />
</target>
</project>

View File

@ -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;
}
private void init(String context, int port) {
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 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);
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,21 +212,24 @@ 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");
@ -153,6 +237,18 @@ public class JettySolrRunner {
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 {

View File

@ -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<String, Slice> sliceMap = state.getSlices(collection);
if (sliceMap == null) {
return "shard1";
}
List<String> shardIdNames = new ArrayList<String>(sliceMap.keySet());
if (shardIdNames.size() < shards) {
return "shard" + (shardIdNames.size() + 1);
}
// else figure out which shard needs more replicas
final Map<String, Integer> map = new HashMap<String, Integer>();
for (String shardId : shardIdNames) {
int cnt = sliceMap.get(shardId).getShards().size();
map.put(shardId, cnt);
}
Collections.sort(shardIdNames, new Comparator<String>() {
@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;
}
}

View File

@ -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,10 +17,13 @@ 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;

View File

@ -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<CoreDescriptor> getCurrentDescriptors();
}

View File

@ -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<String,Slice> slices = cloudState.getSlices(this.collection);
Slice slice = slices.get(shardId);
Map<String,ZkNodeProps> shards = slice.getShards();
boolean foundSomeoneElseActive = false;
for (Map.Entry<String,ZkNodeProps> 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);
}
}

View File

@ -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<String> seqs = zkClient.getChildren(holdElectionPath, null, true);
sortSeqs(seqs);
List<Integer> 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<Integer> getSeqs(List<String> seqs) {
List<Integer> intSeqs = new ArrayList<Integer>(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<String> 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<String> seqs) {
Collections.sort(seqs, new Comparator<String>() {
@Override
public int compare(String o1, String o2) {
return Integer.valueOf(getSeq(o1)).compareTo(
Integer.valueOf(getSeq(o2)));
}
});
}
}

View File

@ -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<CoreState> states)
throws KeeperException, InterruptedException;
}
private final SolrZkClient zkClient;
private final String path;
private volatile Set<CoreState> currentState = new HashSet<CoreState>();
private final NodeStateChangeListener listener;
private final String nodeName;
public Set<CoreState> 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<CoreState> stateList = Arrays.asList(states);
HashSet<CoreState> modifiedCores = new HashSet<CoreState>();
modifiedCores.addAll(stateList);
modifiedCores.removeAll(currentState);
HashSet<CoreState> newState = new HashSet<CoreState>();
newState.addAll(stateList);
HashMap<String, CoreState> lookup = new HashMap<String, CoreState>();
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
}
}
}

View File

@ -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<String,NodeStateWatcher> nodeStateWatches = new HashMap<String,NodeStateWatcher>();
// shard leader watchers (collection->slice->watcher)
private HashMap<String, HashMap<String,ShardLeaderWatcher>> shardLeaderWatches = new HashMap<String,HashMap<String,ShardLeaderWatcher>>();
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<String> collections = zkClient.getChildren(ZkStateReader.COLLECTIONS_ZKNODE, new Watcher(){
@Override
public void process(WatchedEvent event) {
try {
List<String> 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<String> collections) throws KeeperException, InterruptedException {
synchronized (shardLeaderWatches) {
for(String collection: collections) {
if(!shardLeaderWatches.containsKey(collection)) {
shardLeaderWatches.put(collection, new HashMap<String,ShardLeaderWatcher>());
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<String> leaderNodes = zkClient.getChildren(
ZkStateReader.getShardLeadersPath(collection, null), new Watcher() {
@Override
public void process(WatchedEvent event) {
try {
List<String> 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<String> shardIds) {
if(log.isInfoEnabled()) {
log.info("Leader nodes changed for collection: " + collection + " nodes now:" + shardIds);
}
Map<String, ShardLeaderWatcher> watches = shardLeaderWatches.get(collection);
Set<String> currentWatches = new HashSet<String>();
currentWatches.addAll(watches.keySet());
Set<String> newLeaders = complement(shardIds, currentWatches);
Set<String> 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<String> 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<String> liveNodes = zkClient.getChildren(
ZkStateReader.LIVE_NODES_ZKNODE, this, true);
Set<String> liveNodesSet = new HashSet<String>();
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.<String>emptySet(), liveNodes);
}
private void processLiveNodesChanged(Collection<String> oldLiveNodes,
Collection<String> liveNodes) throws InterruptedException, KeeperException {
Set<String> upNodes = complement(liveNodes, oldLiveNodes);
if (upNodes.size() > 0) {
addNodeStateWatches(upNodes);
}
Set<String> downNodes = complement(oldLiveNodes, liveNodes);
for(String node: downNodes) {
NodeStateWatcher watcher = nodeStateWatches.remove(node);
}
}
private void addNodeStateWatches(Set<String> 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<String,String> props = new HashMap<String,String>();
for (Entry<String,String> entry : coreState.getProperties().entrySet()) {
props.put(entry.getKey(), entry.getValue());
}
ZkNodeProps zkProps = new ZkNodeProps(props);
Slice slice = state.getSlice(collection, shardId);
Map<String,ZkNodeProps> shardProps;
if (slice == null) {
shardProps = new HashMap<String,ZkNodeProps>();
} 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<String> complement(Collection<String> next,
Collection<String> prev) {
Set<String> downCollections = new HashSet<String>();
downCollections.addAll(next);
downCollections.removeAll(prev);
return downCollections;
}
@Override
public void coreChanged(final String nodeName, final Set<CoreState> 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<String, Map<String, Slice>> newStates = new LinkedHashMap<String,Map<String,Slice>>();
newStates.putAll(state.getCollectionStates());
if (!newStates.containsKey(collection)) {
newStates.put(collection, new LinkedHashMap<String,Slice>());
}
final Map<String, Slice> slices = newStates.get(collection);
if (!slices.containsKey(slice.getName())) {
slices.put(slice.getName(), slice);
} else {
final Map<String,ZkNodeProps> shards = new LinkedHashMap<String,ZkNodeProps>();
final Slice existingSlice = slices.get(slice.getName());
shards.putAll(existingSlice.getShards());
//XXX preserve existing leader
for(Entry<String, ZkNodeProps> edit: slice.getShards().entrySet()) {
if(existingSlice.getShards().get(edit.getKey())!=null && existingSlice.getShards().get(edit.getKey()).containsKey(ZkStateReader.LEADER_PROP)) {
HashMap<String, String> newProps = new HashMap<String,String>();
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<String, Map<String, Slice>> newStates = new LinkedHashMap<String,Map<String,Slice>>();
newStates.putAll(state.getCollectionStates());
final Map<String, Slice> 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<String,ZkNodeProps> newShards = new LinkedHashMap<String,ZkNodeProps>();
for(Entry<String, ZkNodeProps> shard: slices.get(sliceName).getShards().entrySet()) {
Map<String, String> newShardProps = new LinkedHashMap<String,String>();
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());
}
}
}
}

View File

@ -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<SolrIndexSearcher> 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<RecoveryInfo> replay(UpdateLog ulog)
throws InterruptedException, ExecutionException, TimeoutException {
Future<RecoveryInfo> future = ulog.applyBufferedUpdates();
if (future == null) {
// no replay needed\
log.info("No replay needed");
} else {
// wait for replay
future.get();
}
// solrcloud_debug
// try {
// RefCounted<SolrIndexSearcher> 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;
}
}

View File

@ -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;
}
}

View File

@ -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<String,Slice> slices = cloudState.getSlices(collection);
Slice slice = slices.get(shardId);
Map<String,ZkNodeProps> shards = slice.getShards();
for (Map.Entry<String,ZkNodeProps> 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<ZkCoreNodeProps> 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<String> syncWith = new ArrayList<String>();
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<ZkCoreNodeProps> 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;
}
}

View File

@ -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;
@ -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<String, CoreState> coreStates = Collections.synchronizedMap(new HashMap<String, CoreState>());
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<CoreDescriptor> 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);
@ -304,52 +351,16 @@ 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,21 +380,13 @@ 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() {
@ -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();
boolean shardZkNodeAlreadyExists = zkClient.exists(shardsZkPath);
if(shardZkNodeAlreadyExists && !forcePropsUpdate) {
return;
public String register(String coreName, final CoreDescriptor desc) throws Exception {
return register(coreName, desc, false);
}
/**
* 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();
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<String,String> props = new HashMap<String,String>();
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);
+ 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 shardZkNodeName = getNodeName() + "_" + coreName;
String leaderUrl = zkStateReader.getLeaderUrl(collection,
cloudDesc.getShardId(), 30000);
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);
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
log.info("We are " + ourUrl + " and leader is " + leaderUrl);
boolean isLeader = leaderUrl.equals(ourUrl);
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<UpdateLog.RecoveryInfo> 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<String,String> finalProps = new HashMap<String,String>();
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<String,String> finalProps = new HashMap<String,String>();
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<String,String> finalProps = new HashMap<String,String>();
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<String,String> finalProps = new HashMap<String,String>();
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<String,String> collectionProps = new HashMap<String,String>();
// 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) {
@ -659,8 +765,130 @@ public final class ZkController {
}
private void getConfName(String collection, String collectionPath,
Map<String,String> 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<String> 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<String,String> 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<String,String> 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);
}
}

View File

@ -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<String> 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();
@ -120,4 +120,8 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
return list.toArray(new String[0]);
}
public String getCollectionZkPath() {
return collectionZkPath;
}
}

View File

@ -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) {

View File

@ -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<CoreDescriptor> getCurrentDescriptors() {
List<CoreDescriptor> descriptors = new ArrayList<CoreDescriptor>(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");
}
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; i<nodes.getLength(); i++) {
Node node = nodes.item(i);
try {
@ -374,6 +382,10 @@ public class CoreContainer
if (opt != null) {
p.getCloudDescriptor().setCollectionName(opt);
}
opt = DOMUtil.getAttr(node, "roles", null);
if(opt != null){
p.getCloudDescriptor().setRoles(opt);
}
}
opt = DOMUtil.getAttr(node, "properties", null);
if (opt != null) {
@ -393,29 +405,6 @@ public class CoreContainer
SolrException.log(log,null,ex);
}
}
if(zkController != null) {
try {
synchronized (zkController.getZkStateReader().getUpdateLock()) {
zkController.getZkStateReader().makeShardZkNodeWatches(false);
zkController.getZkStateReader().updateCloudState(true);
}
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (KeeperException e) {
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);
}
}
}
private Properties readProperties(Config cfg, Node node) throws XPathExpressionException {
@ -428,7 +417,13 @@ public class CoreContainer
}
return properties;
}
private boolean isShutDown = false;
private volatile boolean isShutDown = false;
public boolean isShutDown() {
return isShutDown;
}
/**
* Stops all cores.
*/
@ -437,8 +432,14 @@ public class CoreContainer
synchronized(cores) {
try {
for (SolrCore core : cores.values()) {
try {
if (!core.isClosed()) {
core.close();
}
} catch (Throwable t) {
SolrException.log(log, "Error shutting down core", t);
}
}
cores.clear();
} finally {
if(zkController != null) {
@ -447,6 +448,9 @@ public class CoreContainer
if (zkServer != null) {
zkServer.stop();
}
if (shardHandlerFactory != null) {
shardHandlerFactory.close();
}
isShutDown = true;
}
}
@ -457,7 +461,6 @@ public class CoreContainer
try {
if(!isShutDown){
log.error("CoreContainer was not shutdown prior to finalize(), indicates a bug -- POSSIBLE RESOURCE LEAK!!! instance=" + System.identityHashCode(this));
shutdown();
}
} finally {
super.finalize();
@ -480,6 +483,12 @@ public class CoreContainer
throw new RuntimeException( "Invalid core name: "+name );
}
if (zkController != null) {
// before becoming available, make sure we are not live and active
// this also gets us our assigned shard id if it was not specified
zkController.publish(core, ZkStateReader.DOWN);
}
SolrCore old = null;
synchronized (cores) {
old = cores.put(name, core);
@ -491,27 +500,9 @@ public class CoreContainer
core.getCoreDescriptor().name = name;
}
if (zkController != null) {
try {
zkController.register(core.getName(), core.getCoreDescriptor().getCloudDescriptor(), true);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
} catch (IOException e) {
log.error("", e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
if( old == null || old == core) {
log.info( "registering core: "+name );
registerInZk(core);
return null;
}
else {
@ -519,11 +510,34 @@ public class CoreContainer
if (!returnPrevNotClosed) {
old.close();
}
registerInZk(core);
return old;
}
}
private void registerInZk(SolrCore core) {
if (zkController != null) {
try {
zkController.register(core.getName(), core.getCoreDescriptor());
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
} catch (Exception e) {
// if register fails, this is really bad - close the zkController to
// minimize any damage we can cause
zkController.publish(core, ZkStateReader.DOWN);
zkController.close();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
}
}
}
/**
* Registers a SolrCore descriptor in the registry using the core's name.
* If returnPrev==false, the old core, if different, is closed.
@ -562,7 +576,7 @@ public class CoreContainer
try {
String collection = dcore.getCloudDescriptor().getCollectionName();
zkController.createCollectionZkNode(dcore.getCloudDescriptor());
// zkController.createCollectionZkNode(collection);
zkConfigName = zkController.readConfigName(collection);
if (zkConfigName == null) {
log.error("Could not find config name for collection:" + collection);
@ -632,6 +646,12 @@ public class CoreContainer
}
SolrCore core = new SolrCore(dcore.getName(), null, config, schema, dcore);
if (zkController == null && core.getUpdateHandler().getUpdateLog() != null) {
// always kick off recovery if we are in standalone mode.
core.getUpdateHandler().getUpdateLog().recoverFromLog();
}
return core;
}
@ -949,6 +969,22 @@ public class CoreContainer
return zkController;
}
/** The default ShardHandlerFactory used to communicate with other solr instances */
public ShardHandlerFactory getShardHandlerFactory() {
synchronized (this) {
if (shardHandlerFactory == null) {
Map m = new HashMap();
m.put("class",HttpShardHandlerFactory.class.getName());
PluginInfo info = new PluginInfo("shardHandlerFactory", m,null,Collections.<PluginInfo>emptyList());
HttpShardHandlerFactory fac = new HttpShardHandlerFactory();
fac.init(info);
shardHandlerFactory = fac;
}
return shardHandlerFactory;
}
}
private SolrConfig getSolrConfigFromZk(String zkConfigName, String solrConfigFileName,
SolrResourceLoader resourceLoader) throws IOException,
ParserConfigurationException, SAXException, KeeperException,
@ -976,7 +1012,7 @@ public class CoreContainer
private static final String DEF_SOLR_XML ="<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
"<solr persistent=\"false\">\n" +
" <cores adminPath=\"/admin/cores\" defaultCoreName=\"" + DEFAULT_DEFAULT_CORE_NAME + "\">\n" +
" <core name=\""+ DEFAULT_DEFAULT_CORE_NAME + "\" instanceDir=\".\" />\n" +
" <core name=\""+ DEFAULT_DEFAULT_CORE_NAME + "\" shard=\"${shard:}\" instanceDir=\".\" />\n" +
" </cores>\n" +
"</solr>";
}

View File

@ -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) {

View File

@ -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;

View File

@ -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<SolrIndexSearcher> _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<RefCounted<SolrIndexSearcher>> _searchers = new LinkedList<RefCounted<SolrIndexSearcher>>();
private final LinkedList<RefCounted<SolrIndexSearcher>> _realtimeSearchers = new LinkedList<RefCounted<SolrIndexSearcher>>();
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<SolrIndexSearcher> realtimeSearcher;
/**
* Return a registered {@link RefCounted}&lt;{@link SolrIndexSearcher}&gt; with
* the reference count incremented. It <b>must</b> be decremented when no longer needed.
@ -987,30 +981,163 @@ public final class SolrCore implements SolrInfoMBean {
}
/**
* Return the newest {@link RefCounted}&lt;{@link SolrIndexSearcher}&gt; with
* Return the newest normal {@link RefCounted}&lt;{@link SolrIndexSearcher}&gt; with
* the reference count incremented. It <b>must</b> 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<SolrIndexSearcher> 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<SolrIndexSearcher> 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<SolrIndexSearcher> 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<SolrIndexSearcher> getSearcher(boolean forceNew, boolean returnSearcher, final Future[] waitSearcher) throws IOException {
return getSearcher(forceNew, returnSearcher, waitSearcher, false);
}
/** Opens a new searcher and returns a RefCounted<SolrIndexSearcher> 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<SolrIndexSearcher> openNewSearcher(boolean updateHandlerReopens, boolean realtime) {
SolrIndexSearcher tmp;
RefCounted<SolrIndexSearcher> 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<RefCounted<SolrIndexSearcher>> searcherList = realtime ? _realtimeSearchers : _searchers;
RefCounted<SolrIndexSearcher> 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.
* <p>
@ -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<SolrIndexSearcher> 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<SolrIndexSearcher> currSearcherHolder=null;
final RefCounted<SolrIndexSearcher> 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<SolrIndexSearcher> currSearcherHolder = null; // searcher we are autowarming from
RefCounted<SolrIndexSearcher> 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<SolrIndexSearcher> 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,49 +1272,26 @@ 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);
SolrException.log(log,e);
}
return null;
}
}
);
} catch(Exception e) {
// if submit fails, newSearchHolder does not get decref'd
if (newSearchHolder != null) {
newSearchHolder.decref();
if (returnSearcher) {
newSearchHolder.decref();
}
}
throw e;
}
}
if (currSearcher==null && firstSearcherListeners.size() > 0) {
try {
future = searcherExecutor.submit(
new Callable() {
public Object call() throws Exception {
@ -1254,20 +1306,9 @@ public final class SolrCore implements SolrInfoMBean {
}
}
);
} catch(Exception e) {
// if submit fails, newSearchHolder does not get decref'd
if (newSearchHolder != null) {
newSearchHolder.decref();
if (returnSearcher) {
newSearchHolder.decref();
}
}
throw e;
}
}
if (currSearcher!=null && newSearcherListeners.size() > 0) {
try {
future = searcherExecutor.submit(
new Callable() {
public Object call() throws Exception {
@ -1282,34 +1323,21 @@ public final class SolrCore implements SolrInfoMBean {
}
}
);
} catch(Exception e) {
// if submit fails, newSearchHolder does not get decref'd
if (newSearchHolder != null) {
newSearchHolder.decref();
if (returnSearcher) {
newSearchHolder.decref();
}
}
throw e;
}
}
// WARNING: this code assumes a single threaded executor (that all tasks
// queued will finish first).
final RefCounted<SolrIndexSearcher> 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 will decrement onDeckSearchers and
// do a notify, even if it fails.
registerSearcher(newSearchHolder);
} catch (Throwable e) {
SolrException.log(log, null, e);
SolrException.log(log, e);
} finally {
// we are all done with the old searcher we used
// for warming...
@ -1319,34 +1347,27 @@ public final class SolrCore implements SolrInfoMBean {
}
}
);
} catch(Exception e) {
// if submit fails, newSearchHolder does not get decref'd
if (newSearchHolder != null) {
newSearchHolder.decref();
if (returnSearcher) {
newSearchHolder.decref();
}
}
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 {
if (!success) {
synchronized (searcherLock) {
if (decrementOnDeckCount[0]) {
onDeckSearchers--;
}
if (onDeckSearchers < 0) {
// sanity check... should never happen
log.error(logid+"ERROR!!! onDeckSearchers after decrement=" + onDeckSearchers);
@ -1356,15 +1377,28 @@ public final class SolrCore implements SolrInfoMBean {
searcherLock.notify();
}
// since the indexreader was already opened, assume we can continue on
// even though we got an exception.
return returnSearcher ? newSearchHolder : null;
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
}
}
}
// 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<SolrIndexSearcher> newHolder(SolrIndexSearcher newSearcher) {
private RefCounted<SolrIndexSearcher> newHolder(SolrIndexSearcher newSearcher, final List<RefCounted<SolrIndexSearcher>> searcherList) {
RefCounted<SolrIndexSearcher> holder = new RefCounted<SolrIndexSearcher>(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,6 +1478,10 @@ 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
@ -1579,8 +1623,10 @@ public final class SolrCore implements SolrInfoMBean {
}
log.info("created "+info.name+": " + info.className);
} catch (Exception ex) {
throw new SolrException
SolrException e = new SolrException
(SolrException.ErrorCode.SERVER_ERROR, "QueryResponseWriter init failure", ex);
SolrException.log(log,null,e);
throw e;
}
}

View File

@ -140,6 +140,7 @@ public class BinaryUpdateRequestHandler extends ContentStreamHandlerBase {
}
}
}
@Override
public String getDescription() {
return "Add/Update multiple documents with javabin format";

View File

@ -163,10 +163,10 @@ class JsonLoader extends ContentStreamLoader {
String key = parser.getString();
if( parser.wasKey() ) {
if( "id".equals( key ) ) {
cmd.id = parser.getString();
cmd.setId(parser.getString());
}
else if( "query".equals(key) ) {
cmd.query = parser.getString();
cmd.setQuery(parser.getString());
}
else if( "commitWithin".equals(key) ) {
cmd.commitWithin = Integer.parseInt(parser.getString());
@ -181,7 +181,7 @@ class JsonLoader extends ContentStreamLoader {
}
}
else if( ev == JSONParser.OBJECT_END ) {
if( cmd.id == null && cmd.query == null ) {
if( cmd.getId() == null && cmd.getQuery() == null ) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Missing id or query for delete ["+parser.getPosition()+"]" );
}
return cmd;

View File

@ -43,6 +43,7 @@ import org.apache.commons.io.IOUtils;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexDeletionPolicy;
import org.apache.lucene.index.IndexReader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
@ -81,6 +82,7 @@ import org.slf4j.LoggerFactory;
* @since solr 1.4
*/
public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAware {
private static final Logger LOG = LoggerFactory.getLogger(ReplicationHandler.class.getName());
SolrCore core;
@ -128,6 +130,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
// It gives the current 'replicateable' index version
if (command.equals(CMD_INDEX_VERSION)) {
IndexCommit commitPoint = indexCommitPoint; // make a copy so it won't change
//System.out.println("The latest index gen is:" + commitPoint.getGeneration() + " " + core.getCoreDescriptor().getCoreContainer().getZkController().getNodeName());
if (commitPoint != null && replicationEnabled.get()) {
//
// There is a race condition here. The commit point may be changed / deleted by the time
@ -162,7 +166,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
new Thread() {
@Override
public void run() {
doFetch(paramsCopy);
doFetch(paramsCopy, false);
}
}.start();
rsp.add(STATUS, OK_STATUS);
@ -270,10 +274,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
private volatile SnapPuller tempSnapPuller;
void doFetch(SolrParams solrParams) {
public boolean doFetch(SolrParams solrParams, boolean force) {
String masterUrl = solrParams == null ? null : solrParams.get(MASTER_URL);
if (!snapPullLock.tryLock())
return;
return false;
try {
tempSnapPuller = snapPuller;
if (masterUrl != null) {
@ -281,13 +285,14 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
nl.remove(SnapPuller.POLL_INTERVAL);
tempSnapPuller = new SnapPuller(nl, this, core);
}
tempSnapPuller.fetchLatestIndex(core);
return tempSnapPuller.fetchLatestIndex(core, force);
} catch (Exception e) {
LOG.error("SnapPull failed ", e);
SolrException.log(LOG, "SnapPull failed ", e);
} finally {
tempSnapPuller = snapPuller;
snapPullLock.unlock();
}
return false;
}
boolean isReplicating() {
@ -334,6 +339,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
long version = Long.parseLong(v);
IndexCommit commit = core.getDeletionPolicy().getCommitPoint(version);
//System.out.println("ask for files for gen:" + commit.getGeneration() + core.getCoreDescriptor().getCoreContainer().getZkController().getNodeName());
if (commit == null) {
rsp.add("status", "invalid indexversion");
return;
@ -757,12 +764,12 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
void refreshCommitpoint() {
IndexCommit commitPoint = core.getDeletionPolicy().getLatestCommit();
if(replicateOnCommit || (replicateOnOptimize && commitPoint.getSegmentCount() == 1)) {
indexCommitPoint = commitPoint;
}
}
// void refreshCommitpoint() {
// IndexCommit commitPoint = core.getDeletionPolicy().getLatestCommit();
// if(replicateOnCommit || (replicateOnOptimize && commitPoint.getSegmentCount() == 1)) {
// indexCommitPoint = commitPoint;
// }
// }
@SuppressWarnings("unchecked")
public void inform(SolrCore core) {
@ -777,6 +784,12 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
NamedList master = (NamedList) initArgs.get("master");
boolean enableMaster = isEnabled( master );
if (!enableSlave && !enableMaster) {
enableMaster = true;
master = new NamedList<Object>();
}
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) {

View File

@ -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<SolrIndexSearcher> 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<Map<String, Object>>());
// if the generateion of master is older than that of the slave , it means they are not compatible to be copied
// then a new index direcory to be created and all the files need to be copied
boolean isFullCopyNeeded = commit.getGeneration() >= latestGeneration;
boolean isFullCopyNeeded = commit.getVersion() >= latestVersion || force;
File tmpIndexDir = createTempindexDir(core);
if (isIndexStale())
isFullCopyNeeded = true;
@ -331,15 +351,17 @@ public class SnapPuller {
return successfulInstall;
} catch (ReplicationHandlerException e) {
LOG.error("User aborted Replication");
return false;
} catch (SolrException e) {
throw e;
} catch (InterruptedException e) {
throw new InterruptedException("Index fetch interrupted");
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Index fetch failed : ", e);
} finally {
if (deleteTmpIdxDir) delTree(tmpIndexDir);
else delTree(indexDir);
}
return successfulInstall;
} finally {
if (!successfulInstall) {
logReplicationTimeAndConfFiles(null, successfulInstall);
@ -476,9 +498,9 @@ public class SnapPuller {
// reboot the writer on the new index and get a new searcher
solrCore.getUpdateHandler().newIndexWriter();
solrCore.getSearcher(true, false, null);
// update our commit point to the right dir
solrCore.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
replicationHandler.refreshCommitpoint();
} finally {
req.close();
}

View File

@ -211,14 +211,24 @@ class XMLLoader extends ContentStreamLoader {
"unexpected XML tag /delete/" + mode);
}
text.setLength(0);
if ("id".equals(mode)) {
for (int i = 0; i < parser.getAttributeCount(); i++) {
String attrName = parser.getAttributeLocalName(i);
String attrVal = parser.getAttributeValue(i);
if (XmlUpdateRequestHandler.VERSION.equals(attrName)) {
deleteCmd.setVersion(Long.parseLong(attrVal));
}
}
}
break;
case XMLStreamConstants.END_ELEMENT:
String currTag = parser.getLocalName();
if ("id".equals(currTag)) {
deleteCmd.id = text.toString();
deleteCmd.setId(text.toString());
} else if ("query".equals(currTag)) {
deleteCmd.query = text.toString();
deleteCmd.setQuery(text.toString());
} else if ("delete".equals(currTag)) {
return;
} else {

View File

@ -44,9 +44,12 @@ public class XmlUpdateRequestHandler extends ContentStreamHandlerBase {
public static final String OVERWRITE = "overwrite";
public static final String VERSION = "version";
// NOTE: This constant is for use with the <add> XML tag, not the HTTP param with same name
public static final String COMMIT_WITHIN = "commitWithin";
XMLInputFactory inputFactory;

View File

@ -17,39 +17,56 @@
package org.apache.solr.handler.admin;
import org.apache.commons.io.FileUtils;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.*;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.util.NumberUtils;
import org.apache.solr.util.RefCounted;
import org.apache.solr.util.SolrPluginUtils;
import org.apache.solr.update.MergeIndexesCommand;
import org.apache.solr.update.processor.UpdateRequestProcessor;
import org.apache.solr.update.processor.UpdateRequestProcessorChain;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;
import org.apache.commons.io.FileUtils;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
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.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.CloseHook;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.SolrCore;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.MergeIndexesCommand;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.solr.update.processor.UpdateRequestProcessor;
import org.apache.solr.update.processor.UpdateRequestProcessorChain;
import org.apache.solr.util.NumberUtils;
import org.apache.solr.util.RefCounted;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
*
* @since solr 1.3
@ -161,6 +178,21 @@ public class CoreAdminHandler extends RequestHandlerBase {
break;
}
case PREPRECOVERY: {
this.handlePrepRecoveryAction(req, rsp);
break;
}
case REQUESTRECOVERY: {
this.handleRequestRecoveryAction(req, rsp);
break;
}
case DISTRIBURL: {
this.handleDistribUrlAction(req, rsp);
break;
}
default: {
doPersist = this.handleCustomAction(req, rsp);
break;
@ -555,6 +587,142 @@ public class CoreAdminHandler extends RequestHandlerBase {
}
protected void handleRequestRecoveryAction(SolrQueryRequest req,
SolrQueryResponse rsp) throws IOException {
final SolrParams params = req.getParams();
String cname = params.get(CoreAdminParams.CORE);
if (cname == null) {
cname = "";
}
SolrCore core = null;
try {
core = coreContainer.getCore(cname);
core.getUpdateHandler().getSolrCoreState().doRecovery(core);
} finally {
// no recoveryStrat close for now
if (core != null) {
core.close();
}
}
}
protected void handlePrepRecoveryAction(SolrQueryRequest req,
SolrQueryResponse rsp) throws IOException, InterruptedException {
final SolrParams params = req.getParams();
String cname = params.get(CoreAdminParams.CORE);
if (cname == null) {
cname = "";
}
String nodeName = params.get("nodeName");
String coreNodeName = params.get("coreNodeName");
SolrCore core = null;
try {
core = coreContainer.getCore(cname);
if (core == null) {
throw new SolrException(ErrorCode.BAD_REQUEST, "core not found:" + cname);
}
String state;
int retry = 0;
while (true) {
// wait until we are sure the recovering node is ready
// to accept updates
CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
.getCloudDescriptor();
CloudState cloudState = coreContainer
.getZkController()
.getCloudState();
ZkNodeProps nodeProps =
cloudState.getSlice(cloudDescriptor.getCollectionName(),
cloudDescriptor.getShardId()).getShards().get(coreNodeName);
state = nodeProps.get(ZkStateReader.STATE_PROP);
boolean live = cloudState.liveNodesContain(nodeName);
if (nodeProps != null && state.equals(ZkStateReader.RECOVERING)
&& live) {
break;
}
if (retry++ == 30) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"I was asked to prep for recovery for " + nodeName
+ " but she is not live or not in a recovery state - state: " + state + " live:" + live);
}
Thread.sleep(1000);
}
// small safety net for any updates that started with state that
// kept it from sending the update to be buffered -
// pause for a while to let any outstanding updates finish
Thread.sleep(4000);
UpdateRequestProcessorChain processorChain = core
.getUpdateProcessingChain(params.get(UpdateParams.UPDATE_CHAIN));
ModifiableSolrParams reqParams = new ModifiableSolrParams(req.getParams());
reqParams.set(DistributedUpdateProcessor.COMMIT_END_POINT, "true");
SolrQueryRequest sqr = new LocalSolrQueryRequest(core, reqParams);
UpdateRequestProcessor processor = processorChain.createProcessor(sqr,
new SolrQueryResponse());
CommitUpdateCommand cuc = new CommitUpdateCommand(req, false);
processor.processCommit(cuc);
processor.finish();
// solrcloud_debug
// try {
// RefCounted<SolrIndexSearcher> searchHolder = core.getNewestSearcher(false);
// SolrIndexSearcher searcher = searchHolder.get();
// try {
// System.out.println(core.getCoreDescriptor().getCoreContainer().getZkController().getNodeName() + " to replicate "
// + searcher.search(new MatchAllDocsQuery(), 1).totalHits + " gen:" + core.getDeletionPolicy().getLatestCommit().getGeneration() + " data:" + core.getDataDir());
// } finally {
// searchHolder.decref();
// }
// } catch (Exception e) {
//
// }
} finally {
if (core != null) {
core.close();
}
}
}
protected void handleDistribUrlAction(SolrQueryRequest req,
SolrQueryResponse rsp) throws IOException, InterruptedException, SolrServerException {
// TODO: finish this and tests
SolrParams params = req.getParams();
SolrParams required = params.required();
String path = required.get("path");
String shard = params.get("shard");
String collection = required.get("collection");
SolrCore core = req.getCore();
ZkController zkController = core.getCoreDescriptor().getCoreContainer()
.getZkController();
if (shard != null) {
List<ZkCoreNodeProps> replicas = zkController.getZkStateReader().getReplicaProps(
collection, shard, zkController.getNodeName(), core.getName());
for (ZkCoreNodeProps node : replicas) {
CommonsHttpSolrServer server = new CommonsHttpSolrServer(node.getCoreUrl() + path);
QueryRequest qr = new QueryRequest();
server.request(qr);
}
}
}
protected NamedList<Object> getCoreStatus(CoreContainer cores, String cname) throws IOException {
NamedList<Object> info = new SimpleOrderedMap<Object>();
SolrCore core = cores.getCore(cname);
@ -594,6 +762,13 @@ public class CoreAdminHandler extends RequestHandlerBase {
return path;
}
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;
}
//////////////////////// SolrInfoMBeans methods //////////////////////

View File

@ -18,26 +18,32 @@
package org.apache.solr.handler.admin;
import org.apache.commons.io.IOUtils;
import org.apache.solr.cloud.ZkSolrResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ContentStreamBase;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.RawResponseWriter;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.zookeeper.KeeperException;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.UnsupportedEncodingException;
import java.net.URISyntaxException;
import java.util.Date;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Set;
@ -111,8 +117,90 @@ public class ShowFileRequestHandler extends RequestHandlerBase
}
@Override
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, KeeperException, InterruptedException
{
CoreContainer coreContainer = req.getCore().getCoreDescriptor().getCoreContainer();
if (coreContainer.isZooKeeperAware()) {
showFromZooKeeper(req, rsp, coreContainer);
} else {
showFromFileSystem(req, rsp);
}
}
private void showFromZooKeeper(SolrQueryRequest req, SolrQueryResponse rsp,
CoreContainer coreContainer) throws KeeperException,
InterruptedException, UnsupportedEncodingException {
String adminFile = null;
SolrCore core = req.getCore();
SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
final ZkSolrResourceLoader loader = (ZkSolrResourceLoader) core
.getResourceLoader();
String confPath = loader.getCollectionZkPath();
String fname = req.getParams().get("file", null);
if (fname == null) {
adminFile = confPath;
} else {
fname = fname.replace('\\', '/'); // normalize slashes
if (hiddenFiles.contains(fname.toUpperCase(Locale.ENGLISH))) {
throw new SolrException(ErrorCode.FORBIDDEN, "Can not access: " + fname);
}
if (fname.indexOf("..") >= 0) {
throw new SolrException(ErrorCode.FORBIDDEN, "Invalid path: " + fname);
}
adminFile = confPath + "/" + fname;
}
// Make sure the file exists, is readable and is not a hidden file
if (!zkClient.exists(adminFile, true)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Can not find: "
+ adminFile);
}
// Show a directory listing
List<String> children = zkClient.getChildren(adminFile, null, true);
if (children.size() > 0) {
NamedList<SimpleOrderedMap<Object>> files = new SimpleOrderedMap<SimpleOrderedMap<Object>>();
for (String f : children) {
if (hiddenFiles.contains(f.toUpperCase(Locale.ENGLISH))) {
continue; // don't show 'hidden' files
}
if (f.startsWith(".")) {
continue; // skip hidden system files...
}
SimpleOrderedMap<Object> fileInfo = new SimpleOrderedMap<Object>();
files.add(f, fileInfo);
List<String> fchildren = zkClient.getChildren(adminFile, null, true);
if (fchildren.size() > 0) {
fileInfo.add("directory", true);
} else {
// TODO? content type
fileInfo.add("size", f.length());
}
// TODO: ?
// fileInfo.add( "modified", new Date( f.lastModified() ) );
}
rsp.add("files", files);
} else {
// Include the file contents
// The file logic depends on RawResponseWriter, so force its use.
ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
params.set(CommonParams.WT, "raw");
req.setParams(params);
ContentStreamBase content = new ContentStreamBase.StringStream(
new String(zkClient.getData(adminFile, null, null, true), "UTF-8"));
content.setContentType(req.getParams().get(USE_CONTENT_TYPE));
rsp.add(RawResponseWriter.CONTENT, content);
}
rsp.setHttpCaching(false);
}
private void showFromFileSystem(SolrQueryRequest req, SolrQueryResponse rsp)
throws IOException {
File adminFile = null;
final SolrResourceLoader loader = req.getCore().getResourceLoader();

View File

@ -22,11 +22,13 @@ import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.impl.LBHttpSolrServer;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
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.CommonParams;
@ -234,7 +236,8 @@ public class HttpShardHandler extends ShardHandler {
SolrQueryRequest req = rb.req;
SolrParams params = req.getParams();
rb.isDistrib = params.getBool("distrib",false);
rb.isDistrib = params.getBool("distrib", req.getCore().getCoreDescriptor()
.getCoreContainer().isZooKeeperAware());
String shards = params.get(ShardParams.SHARDS);
// for back compat, a shards param with URLs like localhost:8983/solr will mean that this
@ -272,11 +275,36 @@ public class HttpShardHandler extends ShardHandler {
cloudState = zkController.getCloudState();
// TODO: check "collection" for which collection(s) to search.. but for now, just default
// to the collection for this core.
// This can be more efficient... we only record the name, even though we have the
// shard info we need in the next step of mapping slice->shards
// This can be more efficient... we only record the name, even though we
// have the shard info we need in the next step of mapping slice->shards
// Stores the comma-separated list of specified collections.
// Eg: "collection1,collection2,collection3"
String collections = params.get("collection");
if (collections != null) {
// If there were one or more collections specified in the query, split
// each parameter and store as a seperate member of a List.
List<String> collectionList = StrUtils.splitSmart(collections, ",",
true);
// First create an empty HashMap to add the slice info to.
slices = new HashMap<String,Slice>();
// In turn, retrieve the slices that cover each collection from the
// cloud state and add them to the Map 'slices'.
for (int i = 0; i < collectionList.size(); i++) {
String collection = collectionList.get(i);
ClientUtils.appendMap(collection, slices, cloudState.getSlices(collection));
}
} else {
// If no collections were specified, default to the collection for
// this core.
slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
}
// Store the logical slices in the ResponseBuilder and create a new
// String array to hold the physical shards (which will be mapped
// later).
rb.slices = slices.keySet().toArray(new String[slices.size()]);
rb.shards = new String[rb.slices.length];
@ -317,14 +345,16 @@ public class HttpShardHandler extends ShardHandler {
StringBuilder sliceShardsStr = new StringBuilder();
boolean first = true;
for (ZkNodeProps nodeProps : sliceShards.values()) {
if (!liveNodes.contains(nodeProps.get(ZkStateReader.NODE_NAME)))
continue;
ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(nodeProps);
if (!liveNodes.contains(coreNodeProps.getNodeName())
|| !coreNodeProps.getState().equals(
ZkStateReader.ACTIVE)) continue;
if (first) {
first = false;
} else {
sliceShardsStr.append('|');
}
String url = nodeProps.get("url");
String url = coreNodeProps.getCoreUrl();
if (url.startsWith("http://"))
url = url.substring(7);
sliceShardsStr.append(url);

View File

@ -16,6 +16,13 @@ package org.apache.solr.handler.component;
* limitations under the License.
*/
import java.net.MalformedURLException;
import java.util.Random;
import java.util.concurrent.Executor;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.httpclient.DefaultHttpMethodRetryHandler;
import org.apache.commons.httpclient.HttpClient;
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
@ -23,17 +30,12 @@ import org.apache.commons.httpclient.params.HttpMethodParams;
import org.apache.solr.client.solrj.impl.LBHttpSolrServer;
import org.apache.solr.common.SolrException;
import org.apache.solr.core.PluginInfo;
import org.apache.solr.core.SolrCore;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.plugin.PluginInfoInitialized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.MalformedURLException;
import java.util.Random;
import java.util.concurrent.Executor;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
public class HttpShardHandlerFactory extends ShardHandlerFactory implements PluginInfoInitialized{
protected static Logger log = LoggerFactory.getLogger(HttpShardHandlerFactory.class);
@ -44,11 +46,12 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements Plug
//
// Consider CallerRuns policy and a lower max threads to throttle
// requests at some point (or should we simply return failure?)
Executor commExecutor = new ThreadPoolExecutor(
ThreadPoolExecutor commExecutor = new ThreadPoolExecutor(
0,
Integer.MAX_VALUE,
5, TimeUnit.SECONDS, // terminate idle threads after 5 sec
new SynchronousQueue<Runnable>() // directly hand off tasks
new SynchronousQueue<Runnable>(), // directly hand off tasks
new DefaultSolrThreadFactory("httpShardExecutor")
);
@ -58,6 +61,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements Plug
int soTimeout = 0; //current default values
int connectionTimeout = 0; //current default values
public String scheme = "http://"; //current default values
private MultiThreadedHttpConnectionManager mgr;
// socket timeout measured in ms, closes a socket if read
// takes longer than x ms to complete. throws
// java.net.SocketTimeoutException: Read timed out exception
@ -97,7 +102,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements Plug
log.info("Setting shard-connection-timeout to: " + connectionTimeout);
}
}
MultiThreadedHttpConnectionManager mgr = new MultiThreadedHttpConnectionManager();
mgr = new MultiThreadedHttpConnectionManager();
mgr.getParams().setDefaultMaxConnectionsPerHost(20);
mgr.getParams().setMaxTotalConnections(10000);
mgr.getParams().setConnectionTimeout(connectionTimeout);
@ -118,4 +123,23 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements Plug
}
}
@Override
public void close() {
try {
mgr.shutdown();
} catch (Throwable e) {
SolrException.log(log, e);
}
try {
loadbalancer.shutdown();
} catch (Throwable e) {
SolrException.log(log, e);
}
try {
commExecutor.shutdownNow();
} catch (Throwable e) {
SolrException.log(log, e);
}
}
}

View File

@ -31,6 +31,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrDocument;
@ -38,6 +39,7 @@ import org.apache.solr.common.SolrDocumentList;
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.*;
@ -206,7 +208,8 @@ public class QueryComponent extends SearchComponent
SolrQueryRequest req = rb.req;
SolrParams params = req.getParams();
rb.isDistrib = params.getBool("distrib",false);
rb.isDistrib = params.getBool("distrib", req.getCore().getCoreDescriptor()
.getCoreContainer().isZooKeeperAware());
String shards = params.get(ShardParams.SHARDS);
// for back compat, a shards param with URLs like localhost:8983/solr will mean that this
@ -244,11 +247,36 @@ public class QueryComponent extends SearchComponent
cloudState = zkController.getCloudState();
// TODO: check "collection" for which collection(s) to search.. but for now, just default
// to the collection for this core.
// This can be more efficient... we only record the name, even though we have the
// shard info we need in the next step of mapping slice->shards
// Stores the comma-separated list of specified collections.
// Eg: "collection1,collection2,collection3"
String collections = params.get("collection");
if (collections != null) {
// If there were one or more collections specified in the query, split
// each parameter and store as a seperate member of a List.
List<String> collectionList = StrUtils.splitSmart(collections, ",",
true);
// First create an empty HashMap to add the slice info to.
slices = new HashMap<String,Slice>();
// In turn, retrieve the slices that cover each collection from the
// cloud state and add them to the Map 'slices'.
for (int i = 0; i < collectionList.size(); i++) {
String collection = collectionList.get(i);
ClientUtils.appendMap(collection, slices, cloudState.getSlices(collection));
}
} else {
// If no collections were specified, default to the collection for
// this core.
slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
}
// Store the logical slices in the ResponseBuilder and create a new
// String array to hold the physical shards (which will be mapped
// later).
rb.slices = slices.keySet().toArray(new String[slices.size()]);
rb.shards = new String[rb.slices.length];
@ -289,14 +317,16 @@ public class QueryComponent extends SearchComponent
StringBuilder sliceShardsStr = new StringBuilder();
boolean first = true;
for (ZkNodeProps nodeProps : sliceShards.values()) {
if (!liveNodes.contains(nodeProps.get(ZkStateReader.NODE_NAME)))
continue;
ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(nodeProps);
if (!liveNodes.contains(coreNodeProps.getNodeName())
|| !coreNodeProps.getState().equals(
ZkStateReader.ACTIVE)) continue;
if (first) {
first = false;
} else {
sliceShardsStr.append('|');
}
String url = nodeProps.get("url");
String url = coreNodeProps.getCoreUrl();
if (url.startsWith("http://"))
url = url.substring(7);
sliceShardsStr.append(url);

View File

@ -38,8 +38,11 @@ import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.ReturnFields;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.update.DocumentBuilder;
import org.apache.solr.update.PeerSync;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.util.RefCounted;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.xml.transform.Transformer;
import java.io.IOException;
@ -47,14 +50,10 @@ import java.net.URL;
import java.util.ArrayList;
import java.util.List;
/**
* TODO!
*
*
* @since solr 1.3
*/
public class RealTimeGetComponent extends SearchComponent
{
public static Logger log = LoggerFactory.getLogger(UpdateLog.class);
public static final String COMPONENT_NAME = "get";
@Override
@ -76,6 +75,18 @@ public class RealTimeGetComponent extends SearchComponent
return;
}
String val = params.get("getVersions");
if (val != null) {
processGetVersions(rb);
return;
}
val = params.get("getUpdates");
if (val != null) {
processGetUpdates(rb);
return;
}
String id[] = params.getParams("id");
String ids[] = params.getParams("ids");
@ -142,7 +153,7 @@ public class RealTimeGetComponent extends SearchComponent
// didn't find it in the update log, so it should be in the newest searcher opened
if (searcher == null) {
searcherHolder = req.getCore().getNewestSearcher(false);
searcherHolder = req.getCore().getRealtimeSearcher();
searcher = searcherHolder.get();
}
@ -247,4 +258,112 @@ public class RealTimeGetComponent extends SearchComponent
public URL[] getDocs() {
return null;
}
///////////////////////////////////////////////////////////////////////////////////
// Returns last versions added to index
///////////////////////////////////////////////////////////////////////////////////
public void processGetVersions(ResponseBuilder rb) throws IOException
{
SolrQueryRequest req = rb.req;
SolrQueryResponse rsp = rb.rsp;
SolrParams params = req.getParams();
if (!params.getBool(COMPONENT_NAME, true)) {
return;
}
int nVersions = params.getInt("getVersions", -1);
if (nVersions == -1) return;
String sync = params.get("sync");
if (sync != null) {
processSync(rb, nVersions, sync);
return;
}
UpdateLog ulog = req.getCore().getUpdateHandler().getUpdateLog();
if (ulog == null) return;
UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates();
try {
rb.rsp.add("versions", recentUpdates.getVersions(nVersions));
} finally {
recentUpdates.close(); // cache this somehow?
}
}
public void processSync(ResponseBuilder rb, int nVersions, String sync) {
List<String> replicas = StrUtils.splitSmart(sync, ",", true);
PeerSync peerSync = new PeerSync(rb.req.getCore(), replicas, nVersions);
boolean success = peerSync.sync();
// TODO: more complex response?
rb.rsp.add("sync", success);
}
public void processGetUpdates(ResponseBuilder rb) throws IOException
{
SolrQueryRequest req = rb.req;
SolrQueryResponse rsp = rb.rsp;
SolrParams params = req.getParams();
if (!params.getBool(COMPONENT_NAME, true)) {
return;
}
String versionsStr = params.get("getUpdates");
if (versionsStr == null) return;
UpdateLog ulog = req.getCore().getUpdateHandler().getUpdateLog();
if (ulog == null) return;
List<String> versions = StrUtils.splitSmart(versionsStr, ",", true);
// TODO: get this from cache instead of rebuilding?
UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates();
List<Object> updates = new ArrayList<Object>(versions.size());
long minVersion = Long.MAX_VALUE;
try {
for (String versionStr : versions) {
long version = Long.parseLong(versionStr);
try {
Object o = recentUpdates.lookup(version);
if (o == null) continue;
if (version > 0) {
minVersion = Math.min(minVersion, version);
}
// TODO: do any kind of validation here?
updates.add(o);
} catch (SolrException e) {
log.warn("Exception reading log for updates", e);
} catch (ClassCastException e) {
log.warn("Exception reading log for updates", e);
}
}
// Must return all delete-by-query commands that occur after the first add requested
// since they may apply.
updates.addAll( recentUpdates.getDeleteByQuery(minVersion));
rb.rsp.add("updates", updates);
} finally {
recentUpdates.close(); // cache this somehow?
}
}
}

View File

@ -23,6 +23,7 @@ import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.util.RTimer;
import org.apache.solr.core.CloseHook;
import org.apache.solr.core.PluginInfo;
import org.apache.solr.core.SolrCore;
import org.apache.solr.handler.RequestHandlerBase;
@ -133,12 +134,20 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
log.info("Adding debug component:" + dbgCmp);
}
if(shfInfo ==null) {
Map m = new HashMap();
m.put("class",HttpShardHandlerFactory.class.getName());
shfInfo = new PluginInfo("shardHandlerFactory", m,null,Collections.<PluginInfo>emptyList());
}
shardHandlerFactory = core.getCoreDescriptor().getCoreContainer().getShardHandlerFactory();
} else {
shardHandlerFactory = core.createInitInstance(shfInfo, ShardHandlerFactory.class, null, null);
}
core.addCloseHook(new CloseHook() {
@Override
public void preClose(SolrCore core) {
shardHandlerFactory.close();
}
@Override
public void postClose(SolrCore core) {
}
});
}
public List<SearchComponent> getComponents() {
return components;
@ -247,7 +256,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
for (String shard : sreq.actualShards) {
ModifiableSolrParams params = new ModifiableSolrParams(sreq.params);
params.remove(ShardParams.SHARDS); // not a top-level request
params.remove("distrib"); // not a top-level request
params.set("distrib", "false"); // not a top-level request
params.remove("indent");
params.remove(CommonParams.HEADER_ECHO_PARAMS);
params.set(ShardParams.IS_SHARD, true); // a sub (shard) request

View File

@ -20,4 +20,6 @@ package org.apache.solr.handler.component;
public abstract class ShardHandlerFactory {
public abstract ShardHandler getShardHandler();
public abstract void close();
}

View File

@ -39,6 +39,7 @@ import org.apache.solr.schema.*;
import org.apache.solr.search.*;
import org.apache.solr.util.BoundedTreeSet;
import org.apache.solr.util.DateMathParser;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.handler.component.ResponseBuilder;
import org.apache.solr.util.LongPriorityQueue;
@ -327,6 +328,7 @@ public class SimpleFacets {
Integer.MAX_VALUE,
10, TimeUnit.SECONDS, // terminate idle threads after 10 sec
new SynchronousQueue<Runnable>() // directly hand off tasks
, new DefaultSolrThreadFactory("facetExectutor")
);
/**

View File

@ -80,43 +80,3 @@ public class FunctionRangeQParserPlugin extends QParserPlugin {
}
// This class works as either a normal constant score query, or as a PostFilter using a collector
class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFilter {
final ValueSourceRangeFilter rangeFilt;
public FunctionRangeQuery(ValueSourceRangeFilter filter) {
super(filter);
this.rangeFilt = filter;
}
@Override
public DelegatingCollector getFilterCollector(IndexSearcher searcher) {
Map fcontext = ValueSource.newContext(searcher);
return new FunctionRangeCollector(fcontext);
}
class FunctionRangeCollector extends DelegatingCollector {
final Map fcontext;
ValueSourceScorer scorer;
int maxdoc;
public FunctionRangeCollector(Map fcontext) {
this.fcontext = fcontext;
}
@Override
public void collect(int doc) throws IOException {
if (doc<maxdoc && scorer.matches(doc)) {
delegate.collect(doc);
}
}
@Override
public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
maxdoc = context.reader.maxDoc();
FunctionValues dv = rangeFilt.getValueSource().getValues(fcontext, context);
scorer = dv.getRangeScorer(context.reader, rangeFilt.getLowerVal(), rangeFilt.getUpperVal(), rangeFilt.isIncludeLower(), rangeFilt.isIncludeUpper());
super.setNextReader(context);
}
}
}

View File

@ -0,0 +1,69 @@
/**
* 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.
*/
package org.apache.solr.search;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.ValueSourceScorer;
import org.apache.lucene.search.IndexSearcher;
import org.apache.solr.search.function.ValueSourceRangeFilter;
import java.io.IOException;
import java.util.Map;
// This class works as either a normal constant score query, or as a PostFilter using a collector
public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFilter {
final ValueSourceRangeFilter rangeFilt;
public FunctionRangeQuery(ValueSourceRangeFilter filter) {
super(filter);
this.rangeFilt = filter;
}
@Override
public DelegatingCollector getFilterCollector(IndexSearcher searcher) {
Map fcontext = ValueSource.newContext(searcher);
return new FunctionRangeCollector(fcontext);
}
class FunctionRangeCollector extends DelegatingCollector {
final Map fcontext;
ValueSourceScorer scorer;
int maxdoc;
public FunctionRangeCollector(Map fcontext) {
this.fcontext = fcontext;
}
@Override
public void collect(int doc) throws IOException {
if (doc<maxdoc && scorer.matches(doc)) {
delegate.collect(doc);
}
}
@Override
public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
maxdoc = context.reader.maxDoc();
FunctionValues dv = rangeFilt.getValueSource().getValues(fcontext, context);
scorer = dv.getRangeScorer(context.reader, rangeFilt.getLowerVal(), rangeFilt.getUpperVal(), rangeFilt.isIncludeLower(), rangeFilt.isIncludeUpper());
super.setNextReader(context);
}
}
}

View File

@ -133,6 +133,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
if (dir instanceof FSDirectory) {
FSDirectory fsDirectory = (FSDirectory) dir;
indexDir = fsDirectory.getDirectory().getAbsolutePath();
} else {
log.warn("WARNING: Directory impl does not support setting indexDir: " + dir.getClass().getName());
}
this.closeReader = closeReader;
@ -569,6 +571,37 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
return id == DocIdSetIterator.NO_MORE_DOCS ? -1 : id;
}
/** lookup the docid by the unique key field, and return the id *within* the leaf reader in the low 32 bits, and the index of the leaf reader in the high 32 bits.
* -1 is returned if not found.
* @lucene.internal
*/
public long lookupId(BytesRef idBytes) throws IOException {
String field = schema.getUniqueKeyField().getName();
final AtomicReaderContext[] leaves = leafContexts;
for (int i=0; i<leaves.length; i++) {
final AtomicReaderContext leaf = leaves[i];
final IndexReader reader = leaf.reader;
final Fields fields = reader.fields();
if (fields == null) continue;
final Bits liveDocs = reader.getLiveDocs();
final DocsEnum docs = reader.termDocsEnum(liveDocs, field, idBytes, false);
if (docs == null) continue;
int id = docs.nextDoc();
if (id == DocIdSetIterator.NO_MORE_DOCS) continue;
assert docs.nextDoc() == DocIdSetIterator.NO_MORE_DOCS;
return (((long)i) << 32) | id;
}
return -1;
}
/**
* Compute and cache the DocSet that matches a query.

View File

@ -62,10 +62,10 @@ public class SolrDispatchFilter implements Filter
{
final Logger log = LoggerFactory.getLogger(SolrDispatchFilter.class);
protected CoreContainer cores;
protected volatile CoreContainer cores;
protected String pathPrefix = null; // strip this from the beginning of a path
protected String abortErrorMessage = null;
protected String solrConfigFilename = null;
protected final Map<SolrConfig, SolrRequestParsers> parsers = new WeakHashMap<SolrConfig, SolrRequestParsers>();
protected final SolrRequestParsers adminRequestParser;
@ -101,6 +101,10 @@ public class SolrDispatchFilter implements Filter
log.info("SolrDispatchFilter.init() done");
}
public CoreContainer getCores() {
return cores;
}
/** Method to override to change how CoreContainer initialization is performed. */
protected CoreContainer.Initializer createInitializer() {
return new CoreContainer.Initializer();
@ -119,6 +123,12 @@ public class SolrDispatchFilter implements Filter
return;
}
if (this.cores == null) {
((HttpServletResponse)response).sendError( 403, "Server is shutting down" );
return;
}
CoreContainer cores = this.cores;
if( request instanceof HttpServletRequest) {
HttpServletRequest req = (HttpServletRequest)request;
HttpServletResponse resp = (HttpServletResponse)response;

View File

@ -45,13 +45,20 @@ public class AddUpdateCommand extends UpdateCommand {
public int commitWithin = -1;
public AddUpdateCommand(SolrQueryRequest req) {
super("add", req);
super(req);
}
@Override
public String name() {
return "add";
}
/** Reset state to reuse this object with a different document in the same request */
public void clear() {
solrDoc = null;
indexedId = null;
updateTerm = null;
version = 0;
}
public SolrInputDocument getSolrInputDocument() {
@ -91,6 +98,10 @@ public class AddUpdateCommand extends UpdateCommand {
return indexedId;
}
public void setIndexedId(BytesRef indexedId) {
this.indexedId = indexedId;
}
public String getPrintableId() {
IndexSchema schema = req.getSchema();
SchemaField sf = schema.getUniqueKeyField();
@ -105,10 +116,11 @@ public class AddUpdateCommand extends UpdateCommand {
@Override
public String toString() {
StringBuilder sb = new StringBuilder(commandName);
sb.append(':');
if (indexedId !=null) sb.append("id=").append(indexedId);
StringBuilder sb = new StringBuilder(super.toString());
if (indexedId != null) sb.append(",id=").append(indexedId);
if (!overwrite) sb.append(",overwrite=").append(overwrite);
if (commitWithin != -1) sb.append(",commitWithin=").append(commitWithin);
sb.append('}');
return sb.toString();
}
}

View File

@ -37,16 +37,21 @@ public class CommitUpdateCommand extends UpdateCommand {
public int maxOptimizeSegments = 1;
public CommitUpdateCommand(SolrQueryRequest req, boolean optimize) {
super("commit", req);
super(req);
this.optimize=optimize;
}
@Override
public String name() {
return "commit";
}
@Override
public String toString() {
return prepareCommit ? "prepareCommit" :
("commit(optimize="+optimize
return super.toString() + ",optimize="+optimize
+",waitSearcher="+waitSearcher
+",expungeDeletes="+expungeDeletes
+",softCommit="+softCommit
+')');
+'}';
}
}

View File

@ -20,14 +20,21 @@ package org.apache.solr.update;
import java.io.IOException;
import org.apache.lucene.index.IndexWriter;
import org.apache.solr.cloud.RecoveryStrategy;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.SolrCore;
public final class DefaultSolrCoreState extends SolrCoreState {
private final Object recoveryLock = new Object();
private int refCnt = 1;
private SolrIndexWriter indexWriter = null;
private DirectoryFactory directoryFactory;
private boolean recoveryRunning;
private RecoveryStrategy recoveryStrat;
private boolean closed = false;
public DefaultSolrCoreState(DirectoryFactory directoryFactory) {
this.directoryFactory = directoryFactory;
}
@ -50,15 +57,24 @@ public final class DefaultSolrCoreState extends SolrCoreState {
}
@Override
public synchronized void decref() throws IOException {
public void decref(IndexWriterCloser closer) throws IOException {
boolean cancelRecovery = false;
synchronized (this) {
refCnt--;
if (refCnt == 0) {
if (indexWriter != null) {
if (closer != null) {
closer.closeWriter(indexWriter);
} else if (indexWriter != null) {
indexWriter.close();
}
directoryFactory.close();
closed = true;
cancelRecovery = true;
}
}
// don't wait for this in the sync block
if (cancelRecovery) cancelRecovery();
}
@Override
public synchronized void incref() {
@ -86,4 +102,42 @@ public final class DefaultSolrCoreState extends SolrCoreState {
return directoryFactory;
}
@Override
public void doRecovery(SolrCore core) {
cancelRecovery();
synchronized (recoveryLock) {
while (recoveryRunning) {
try {
recoveryLock.wait(1000);
} catch (InterruptedException e) {
}
if (closed) return;
}
recoveryStrat = new RecoveryStrategy(core);
recoveryStrat.start();
recoveryRunning = true;
}
}
@Override
public void cancelRecovery() {
synchronized (recoveryLock) {
if (recoveryStrat != null) {
recoveryStrat.close();
try {
recoveryStrat.join();
} catch (InterruptedException e) {
}
recoveryRunning = false;
recoveryLock.notifyAll();
}
}
}
}

View File

@ -18,6 +18,8 @@
package org.apache.solr.update;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.solr.common.SolrInputField;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaField;
@ -28,18 +30,28 @@ import org.apache.solr.schema.SchemaField;
public class DeleteUpdateCommand extends UpdateCommand {
public String id; // external (printable) id, for delete-by-id
public String query; // query string for delete-by-query
private BytesRef indexedId;
public BytesRef indexedId;
public int commitWithin = -1;
public DeleteUpdateCommand(SolrQueryRequest req) {
super("delete", req);
super(req);
}
@Override
public String name() {
return "delete";
}
public boolean isDeleteById() {
return query == null;
}
public void clear() {
id = null;
query = null;
indexedId = null;
version = 0;
}
/** Returns the indexed ID for this delete. The returned BytesRef is retained across multiple calls, and should not be modified. */
@ -55,14 +67,46 @@ public class DeleteUpdateCommand extends UpdateCommand {
return indexedId;
}
public String getId() {
if (id == null && indexedId != null) {
IndexSchema schema = req.getSchema();
SchemaField sf = schema.getUniqueKeyField();
if (sf != null) {
CharsRef ref = new CharsRef();
sf.getType().indexedToReadable(indexedId, ref);
id = ref.toString();
}
}
return id;
}
public String getQuery() {
return query;
}
public void setQuery(String query) {
this.query = query;
}
public void setIndexedId(BytesRef indexedId) {
this.indexedId = indexedId;
this.id = null;
}
public void setId(String id) {
this.id = id;
this.indexedId = null;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(commandName);
sb.append(':');
if (id!=null) sb.append("id=").append(id);
else sb.append("query=`").append(query).append('`');
StringBuilder sb = new StringBuilder(super.toString());
if (id!=null) sb.append(",id=").append(getId());
if (indexedId!=null) sb.append(",indexedId=").append(getId());
if (query != null) sb.append(",query=`").append(query).append('`');
sb.append(",commitWithin=").append(commitWithin);
sb.append('}');
return sb.toString();
}
}

View File

@ -32,6 +32,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queryparser.classic.ParseException;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanClause.Occur;
@ -45,8 +46,11 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.SolrConfig.UpdateHandlerInfo;
import org.apache.solr.core.SolrCore;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.FunctionRangeQuery;
import org.apache.solr.search.QParser;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.QueryUtils;
import org.apache.solr.search.function.ValueSourceRangeFilter;
/**
* TODO: add soft commitWithin support
@ -54,8 +58,8 @@ import org.apache.solr.search.SolrIndexSearcher;
* <code>DirectUpdateHandler2</code> implements an UpdateHandler where documents are added
* directly to the main Lucene index as opposed to adding to a separate smaller index.
*/
public class DirectUpdateHandler2 extends UpdateHandler {
protected SolrCoreState solrCoreState;
public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState.IndexWriterCloser {
protected final SolrCoreState solrCoreState;
protected final Lock commitLock = new ReentrantLock();
// stats
@ -115,8 +119,10 @@ public class DirectUpdateHandler2 extends UpdateHandler {
softCommitTracker = new CommitTracker("Soft", core, softCommitDocsUpperBound, softCommitTimeUpperBound, true, true);
this.ulog = updateHandler.getUpdateLog();
if (this.ulog != null) {
this.ulog.init(this, core);
}
}
private void deleteAll() throws IOException {
SolrCore.log.info(core.getLogId()+"REMOVING ALL DOCUMENTS FROM INDEX");
@ -170,14 +176,17 @@ public class DirectUpdateHandler2 extends UpdateHandler {
// allow duplicates
writer.addDocument(cmd.getLuceneDocument());
}
// Add to the transaction log *after* successfully adding to the index, if there was no error.
// This ordering ensures that if we log it, it's definitely been added to the the index.
// This also ensures that if a commit sneaks in-between, that we know everything in a particular
// log version was definitely committed.
ulog.add(cmd);
if (ulog != null) ulog.add(cmd);
softCommitTracker.addedDocument( -1 ); // TODO: support commitWithin with soft update
if ((cmd.getFlags() & UpdateCommand.IGNORE_AUTOCOMMIT) == 0) {
commitTracker.addedDocument( cmd.commitWithin );
softCommitTracker.addedDocument( -1 ); // TODO: support commitWithin with soft update
}
rc = 1;
} finally {
@ -207,8 +216,9 @@ public class DirectUpdateHandler2 extends UpdateHandler {
writer.deleteDocuments(deleteTerm);
// SolrCore.verbose("deleteDocuments",deleteTerm,"DONE");
ulog.delete(cmd);
if (ulog != null) ulog.delete(cmd);
if ((cmd.getFlags() & UpdateCommand.IGNORE_AUTOCOMMIT) == 0) {
if (commitTracker.getTimeUpperBound() > 0) {
commitTracker.scheduleCommitWithin(commitTracker.getTimeUpperBound());
}
@ -217,6 +227,7 @@ public class DirectUpdateHandler2 extends UpdateHandler {
softCommitTracker.scheduleCommitWithin(softCommitTracker.getTimeUpperBound());
}
}
}
// we don't return the number of docs deleted because it's not always possible to quickly know that info.
@Override
@ -227,8 +238,27 @@ public class DirectUpdateHandler2 extends UpdateHandler {
try {
Query q;
try {
// TODO: move this higher in the stack?
QParser parser = QParser.getParser(cmd.query, "lucene", cmd.req);
q = parser.getQuery();
q = QueryUtils.makeQueryable(q);
// peer-sync can cause older deleteByQueries to be executed and could
// delete newer documents. We prevent this by adding a clause restricting
// version.
if ((cmd.getFlags() & UpdateCommand.PEER_SYNC) != 0) {
BooleanQuery bq = new BooleanQuery();
bq.add(q, Occur.MUST);
SchemaField sf = core.getSchema().getField(VersionInfo.VERSION_FIELD);
ValueSource vs = sf.getType().getValueSource(sf, null);
ValueSourceRangeFilter filt = new ValueSourceRangeFilter(vs, null, Long.toString(Math.abs(cmd.version)), true, true);
FunctionRangeQuery range = new FunctionRangeQuery(filt);
bq.add(range, Occur.MUST);
q = bq;
}
} catch (ParseException e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
@ -237,13 +267,21 @@ public class DirectUpdateHandler2 extends UpdateHandler {
commitTracker.deletedDocument(cmd.commitWithin);
//
// synchronized to prevent deleteByQuery from running during the "open new searcher"
// part of a commit. DBQ needs to signal that a fresh reader will be needed for
// a realtime view of the index. When a new searcher is opened after a DBQ, that
// flag can be cleared. If those thing happen concurrently, it's not thread safe.
//
synchronized (this) {
if (delAll) {
deleteAll();
} else {
solrCoreState.getIndexWriter(core).deleteDocuments(q);
}
ulog.deleteByQuery(cmd);
if (ulog != null) ulog.deleteByQuery(cmd);
}
madeIt = true;
@ -345,7 +383,7 @@ public class DirectUpdateHandler2 extends UpdateHandler {
if (!cmd.softCommit) {
synchronized (this) { // sync is currently needed to prevent preCommit from being called between preSoft and postSoft... see postSoft comments.
ulog.preCommit(cmd);
if (ulog != null) ulog.preCommit(cmd);
}
// SolrCore.verbose("writer.commit() start writer=",writer);
@ -366,21 +404,21 @@ public class DirectUpdateHandler2 extends UpdateHandler {
if (cmd.softCommit) {
// ulog.preSoftCommit();
synchronized (this) {
ulog.preSoftCommit(cmd);
if (ulog != null) ulog.preSoftCommit(cmd);
core.getSearcher(true, false, waitSearcher, true);
ulog.postSoftCommit(cmd);
if (ulog != null) ulog.postSoftCommit(cmd);
}
// ulog.postSoftCommit();
} else {
synchronized (this) {
ulog.preSoftCommit(cmd);
if (ulog != null) ulog.preSoftCommit(cmd);
core.getSearcher(true, false, waitSearcher);
ulog.postSoftCommit(cmd);
if (ulog != null) ulog.postSoftCommit(cmd);
}
ulog.postCommit(cmd); // postCommit currently means new searcher has also been opened
if (ulog != null) ulog.postCommit(cmd); // postCommit currently means new searcher has
// also been opened
}
// reset commit tracking
if (cmd.softCommit) {
@ -417,25 +455,6 @@ public class DirectUpdateHandler2 extends UpdateHandler {
}
}
@Override
public SolrIndexSearcher reopenSearcher(SolrIndexSearcher previousSearcher) throws IOException {
IndexReader currentReader = previousSearcher.getIndexReader();
IndexReader newReader;
IndexWriter writer = solrCoreState.getIndexWriter(core);
// SolrCore.verbose("start reopen from",previousSearcher,"writer=",writer);
newReader = IndexReader.openIfChanged(currentReader, writer, true);
// SolrCore.verbose("reopen result", newReader);
if (newReader == null) {
currentReader.incRef();
newReader = currentReader;
}
return new SolrIndexSearcher(core, schema, "main", newReader, true, true, true, core.getDirectoryFactory());
}
@Override
public void newIndexWriter() throws IOException {
solrCoreState.newIndexWriter(core);
@ -490,12 +509,44 @@ public class DirectUpdateHandler2 extends UpdateHandler {
numDocsPending.set(0);
solrCoreState.decref();
log.info("closed " + this);
solrCoreState.decref(this);
}
public static boolean commitOnClose = true; // TODO: make this a real config option?
// IndexWriterCloser interface method - called from solrCoreState.decref(this)
@Override
public void closeWriter(IndexWriter writer) throws IOException {
commitLock.lock();
try {
if (!commitOnClose) {
if (writer != null) {
writer.rollback();
}
// we shouldn't close the transaction logs either, but leaving them open
// means we can't delete them on windows.
if (ulog != null) ulog.close();
return;
}
if (writer != null) {
writer.close();
}
// if the writer hits an exception, it's OK (and perhaps desirable)
// to not close the ulog?
// Closing the log currently deletes the log file.
// If this changes, we should record this as a "commit".
if (ulog != null) ulog.close();
} finally {
commitLock.unlock();
}
}
/////////////////////////////////////////////////////////////////////
// SolrInfoMBean stuff: Statistics and Module Info
/////////////////////////////////////////////////////////////////////
@ -567,14 +618,15 @@ public class DirectUpdateHandler2 extends UpdateHandler {
return "DirectUpdateHandler2" + getStatistics();
}
public SolrCoreState getIndexWriterProvider() {
@Override
public SolrCoreState getSolrCoreState() {
return solrCoreState;
}
@Override
public void decref() {
try {
solrCoreState.decref();
solrCoreState.decref(this);
} catch (IOException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, "", e);
}

View File

@ -1,645 +0,0 @@
/**
* 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.
*/
package org.apache.solr.update;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.util.FastInputStream;
import org.apache.solr.common.util.FastOutputStream;
import org.apache.solr.common.util.JavaBinCodec;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.PluginInfo;
import org.apache.solr.core.SolrCore;
import java.io.*;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.FileChannel;
import java.util.*;
import java.util.concurrent.atomic.AtomicInteger;
/** @lucene.experimental */
class NullUpdateLog extends UpdateLog {
@Override
public void init(PluginInfo info) {
}
@Override
public void init(UpdateHandler uhandler, SolrCore core) {
}
@Override
public void add(AddUpdateCommand cmd) {
}
@Override
public void delete(DeleteUpdateCommand cmd) {
}
@Override
public void deleteByQuery(DeleteUpdateCommand cmd) {
}
@Override
public void preCommit(CommitUpdateCommand cmd) {
}
@Override
public void postCommit(CommitUpdateCommand cmd) {
}
@Override
public void preSoftCommit(CommitUpdateCommand cmd) {
}
@Override
public void postSoftCommit(CommitUpdateCommand cmd) {
}
@Override
public Object lookup(BytesRef indexedId) {
return null;
}
@Override
public void close() {
}
}
/** @lucene.experimental */
public class FSUpdateLog extends UpdateLog {
public static String TLOG_NAME="tlog";
long id = -1;
private TransactionLog tlog;
private TransactionLog prevTlog;
private Map<BytesRef,LogPtr> map = new HashMap<BytesRef, LogPtr>();
private Map<BytesRef,LogPtr> prevMap; // used while committing/reopening is happening
private Map<BytesRef,LogPtr> prevMap2; // used while committing/reopening is happening
private TransactionLog prevMapLog; // the transaction log used to look up entries found in prevMap
private TransactionLog prevMapLog2; // the transaction log used to look up entries found in prevMap
private String[] tlogFiles;
private File tlogDir;
private Collection<String> globalStrings;
private String dataDir;
private String lastDataDir;
@Override
public void init(PluginInfo info) {
dataDir = (String)info.initArgs.get("dir");
}
public void init(UpdateHandler uhandler, SolrCore core) {
if (dataDir == null || dataDir.length()==0) {
dataDir = core.getDataDir();
}
if (dataDir.equals(lastDataDir)) {
// on a normal reopen, we currently shouldn't have to do anything
return;
}
lastDataDir = dataDir;
tlogDir = new File(dataDir, TLOG_NAME);
tlogDir.mkdirs();
tlogFiles = getLogList(tlogDir);
id = getLastLogId() + 1; // add 1 since we will create a new log for the next update
}
static class LogPtr {
final long pointer;
public LogPtr(long pointer) {
this.pointer = pointer;
}
public String toString() {
return "LogPtr(" + pointer + ")";
}
}
public static String[] getLogList(File directory) {
final String prefix = TLOG_NAME+'.';
String[] names = directory.list(new FilenameFilter() {
public boolean accept(File dir, String name) {
return name.startsWith(prefix);
}
});
Arrays.sort(names);
return names;
}
public long getLastLogId() {
if (id != -1) return id;
if (tlogFiles.length == 0) return -1;
String last = tlogFiles[tlogFiles.length-1];
return Long.parseLong(last.substring(TLOG_NAME.length()+1));
}
@Override
public void add(AddUpdateCommand cmd) {
synchronized (this) {
ensureLog();
long pos = tlog.write(cmd);
LogPtr ptr = new LogPtr(pos);
map.put(cmd.getIndexedId(), ptr);
// SolrCore.verbose("TLOG: added id " + cmd.getPrintableId() + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
}
}
@Override
public void delete(DeleteUpdateCommand cmd) {
BytesRef br = cmd.getIndexedId();
synchronized (this) {
ensureLog();
long pos = tlog.writeDelete(cmd);
LogPtr ptr = new LogPtr(pos);
map.put(br, ptr);
// SolrCore.verbose("TLOG: added delete for id " + cmd.id + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
}
}
@Override
public void deleteByQuery(DeleteUpdateCommand cmd) {
synchronized (this) {
ensureLog();
// TODO: how to support realtime-get, optimistic concurrency, or anything else in this case?
// Maybe we shouldn't?
// realtime-get could just do a reopen of the searcher
// optimistic concurrency? Maybe we shouldn't support deleteByQuery w/ optimistic concurrency
long pos = tlog.writeDeleteByQuery(cmd);
LogPtr ptr = new LogPtr(pos);
// SolrCore.verbose("TLOG: added deleteByQuery " + cmd.query + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
}
}
private void newMap() {
prevMap2 = prevMap;
prevMapLog2 = prevMapLog;
prevMap = map;
prevMapLog = tlog;
map = new HashMap<BytesRef, LogPtr>();
}
private void clearOldMaps() {
prevMap = null;
prevMap2 = null;
}
@Override
public void preCommit(CommitUpdateCommand cmd) {
synchronized (this) {
// since we're changing the log, we must change the map.
newMap();
// since document additions can happen concurrently with commit, create
// a new transaction log first so that we know the old one is definitely
// in the index.
prevTlog = tlog;
tlog = null;
id++;
if (prevTlog != null) {
globalStrings = prevTlog.getGlobalStrings();
}
}
}
@Override
public void postCommit(CommitUpdateCommand cmd) {
synchronized (this) {
if (prevTlog != null) {
prevTlog.decref();
prevTlog = null;
}
}
}
@Override
public void preSoftCommit(CommitUpdateCommand cmd) {
synchronized (this) {
if (!cmd.softCommit) return; // already handled this at the start of the hard commit
newMap();
// start adding documents to a new map since we won't know if
// any added documents will make it into this commit or not.
// But we do know that any updates already added will definitely
// show up in the latest reader after the commit succeeds.
map = new HashMap<BytesRef, LogPtr>();
// SolrCore.verbose("TLOG: preSoftCommit: prevMap="+ System.identityHashCode(prevMap) + " new map=" + System.identityHashCode(map));
}
}
@Override
public void postSoftCommit(CommitUpdateCommand cmd) {
synchronized (this) {
// We can clear out all old maps now that a new searcher has been opened.
// This currently only works since DUH2 synchronizes around preCommit to avoid
// it being called in the middle of a preSoftCommit, postSoftCommit sequence.
// If this DUH2 synchronization were to be removed, preSoftCommit should
// record what old maps were created and only remove those.
clearOldMaps();
// SolrCore.verbose("TLOG: postSoftCommit: disposing of prevMap="+ System.identityHashCode(prevMap));
}
}
@Override
public Object lookup(BytesRef indexedId) {
LogPtr entry;
TransactionLog lookupLog;
synchronized (this) {
entry = map.get(indexedId);
lookupLog = tlog; // something found in "map" will always be in "tlog"
// SolrCore.verbose("TLOG: lookup: for id " + indexedId.utf8ToString() + " in map " + System.identityHashCode(map) + " got " + entry + " lookupLog=" + lookupLog);
if (entry == null && prevMap != null) {
entry = prevMap.get(indexedId);
// something found in prevMap will always be found in preMapLog (which could be tlog or prevTlog)
lookupLog = prevMapLog;
// SolrCore.verbose("TLOG: lookup: for id " + indexedId.utf8ToString() + " in prevMap " + System.identityHashCode(prevMap) + " got " + entry + " lookupLog="+lookupLog);
}
if (entry == null && prevMap2 != null) {
entry = prevMap2.get(indexedId);
// something found in prevMap2 will always be found in preMapLog2 (which could be tlog or prevTlog)
lookupLog = prevMapLog2;
// SolrCore.verbose("TLOG: lookup: for id " + indexedId.utf8ToString() + " in prevMap2 " + System.identityHashCode(prevMap) + " got " + entry + " lookupLog="+lookupLog);
}
if (entry == null) {
return null;
}
lookupLog.incref();
}
try {
// now do the lookup outside of the sync block for concurrency
return lookupLog.lookup(entry.pointer);
} finally {
lookupLog.decref();
}
}
private void ensureLog() {
if (tlog == null) {
String newLogName = String.format("%s.%019d", TLOG_NAME, id);
tlog = new TransactionLog(new File(tlogDir, newLogName), globalStrings);
}
}
@Override
public void close() {
synchronized (this) {
if (prevTlog != null) {
prevTlog.decref();
}
if (tlog != null) {
tlog.decref();
}
}
}
}
/**
* Log Format: List{Operation, Version, ...}
* ADD, VERSION, DOC
* DELETE, VERSION, ID_BYTES
* DELETE_BY_QUERY, VERSION, String
*
* TODO: keep two files, one for [operation, version, id] and the other for the actual
* document data. That way we could throw away document log files more readily
* while retaining the smaller operation log files longer (and we can retrieve
* the stored fields from the latest documents from the index).
*
* This would require keeping all source fields stored of course.
*
* This would also allow to not log document data for requests with commit=true
* in them (since we know that if the request succeeds, all docs will be committed)
*
*/
class TransactionLog {
long id;
File tlogFile;
RandomAccessFile raf;
FileChannel channel;
OutputStream os;
FastOutputStream fos;
InputStream is;
volatile boolean deleteOnClose = true; // we can delete old tlogs since they are currently only used for real-time-get (and in the future, recovery)
AtomicInteger refcount = new AtomicInteger(1);
Map<String,Integer> globalStringMap = new HashMap<String, Integer>();
List<String> globalStringList = new ArrayList<String>();
// write a BytesRef as a byte array
JavaBinCodec.ObjectResolver resolver = new JavaBinCodec.ObjectResolver() {
@Override
public Object resolve(Object o, JavaBinCodec codec) throws IOException {
if (o instanceof BytesRef) {
BytesRef br = (BytesRef)o;
codec.writeByteArray(br.bytes, br.offset, br.length);
return null;
}
return o;
}
};
public class LogCodec extends JavaBinCodec {
public LogCodec() {
super(resolver);
}
@Override
public void writeExternString(String s) throws IOException {
if (s == null) {
writeTag(NULL);
return;
}
// no need to synchronize globalStringMap - it's only updated before the first record is written to the log
Integer idx = globalStringMap.get(s);
if (idx == null) {
// write a normal string
writeStr(s);
} else {
// write the extern string
writeTag(EXTERN_STRING, idx);
}
}
@Override
public String readExternString(FastInputStream fis) throws IOException {
int idx = readSize(fis);
if (idx != 0) {// idx != 0 is the index of the extern string
// no need to synchronize globalStringList - it's only updated before the first record is written to the log
return globalStringList.get(idx - 1);
} else {// idx == 0 means it has a string value
// this shouldn't happen with this codec subclass.
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Corrupt transaction log");
}
}
}
public long writeData(Object o) {
LogCodec codec = new LogCodec();
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
codec.init(fos);
codec.writeVal(o);
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
TransactionLog(File tlogFile, Collection<String> globalStrings) {
try {
this.tlogFile = tlogFile;
raf = new RandomAccessFile(this.tlogFile, "rw");
long start = raf.length();
assert start==0;
if (start > 0) {
raf.setLength(0);
start = 0;
}
// System.out.println("###start= "+start);
channel = raf.getChannel();
os = Channels.newOutputStream(channel);
fos = FastOutputStream.wrap(os);
addGlobalStrings(globalStrings);
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
private void addGlobalStrings(Collection<String> strings) {
if (strings == null) return;
int origSize = globalStringMap.size();
for (String s : strings) {
Integer idx = null;
if (origSize > 0) {
idx = globalStringMap.get(s);
}
if (idx != null) continue; // already in list
globalStringList.add(s);
globalStringMap.put(s, globalStringList.size());
}
assert globalStringMap.size() == globalStringList.size();
}
Collection<String> getGlobalStrings() {
synchronized (fos) {
return new ArrayList<String>(globalStringList);
}
}
private void writeLogHeader(LogCodec codec) throws IOException {
NamedList header = new NamedList<Object>();
header.add("SOLR_TLOG",1); // a magic string + version number?
header.add("strings",globalStringList);
codec.marshal(header, fos);
}
public long write(AddUpdateCommand cmd) {
LogCodec codec = new LogCodec();
synchronized (fos) {
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
SolrInputDocument sdoc = cmd.getSolrInputDocument();
if (pos == 0) { // TODO: needs to be changed if we start writing a header first
addGlobalStrings(sdoc.getFieldNames());
pos = fos.size();
}
/***
System.out.println("###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
if (pos != fos.size()) {
throw new RuntimeException("ERROR" + "###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
}
***/
codec.init(fos);
codec.writeTag(JavaBinCodec.ARR, 3);
codec.writeInt(UpdateLog.ADD); // should just take one byte
codec.writeLong(0); // the version... should also just be one byte if 0
codec.writeSolrInputDocument(cmd.getSolrInputDocument());
// fos.flushBuffer(); // flush later
assert pos < fos.size();
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
}
public long writeDelete(DeleteUpdateCommand cmd) {
LogCodec codec = new LogCodec();
synchronized (fos) {
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
if (pos == 0) {
writeLogHeader(codec);
pos = fos.size();
}
codec.init(fos);
codec.writeTag(JavaBinCodec.ARR, 3);
codec.writeInt(UpdateLog.DELETE); // should just take one byte
codec.writeLong(0); // the version... should also just be one byte if 0
BytesRef br = cmd.getIndexedId();
codec.writeByteArray(br.bytes, br.offset, br.length);
// fos.flushBuffer(); // flush later
assert pos < fos.size();
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
}
public long writeDeleteByQuery(DeleteUpdateCommand cmd) {
LogCodec codec = new LogCodec();
synchronized (fos) {
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
if (pos == 0) {
writeLogHeader(codec);
pos = fos.size();
}
codec.init(fos);
codec.writeTag(JavaBinCodec.ARR, 3);
codec.writeInt(UpdateLog.DELETE_BY_QUERY); // should just take one byte
codec.writeLong(0); // the version... should also just be one byte if 0
codec.writeStr(cmd.query);
// fos.flushBuffer(); // flush later
assert pos < fos.size();
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
}
/* This method is thread safe */
public Object lookup(long pos) {
try {
// make sure any unflushed buffer has been flushed
synchronized (fos) {
// TODO: optimize this by keeping track of what we have flushed up to
fos.flushBuffer();
/***
System.out.println("###flushBuffer to " + fos.size() + " raf.length()=" + raf.length() + " pos="+pos);
if (fos.size() != raf.length() || pos >= fos.size() ) {
throw new RuntimeException("ERROR" + "###flushBuffer to " + fos.size() + " raf.length()=" + raf.length() + " pos="+pos);
}
***/
assert pos < fos.size();
assert fos.size() == channel.size();
}
ChannelFastInputStream fis = new ChannelFastInputStream(channel, pos);
LogCodec codec = new LogCodec();
return codec.readVal(fis);
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
public void incref() {
refcount.incrementAndGet();
}
public void decref() {
if (refcount.decrementAndGet() == 0) {
close();
}
}
private void close() {
try {
fos.flush();
fos.close();
if (deleteOnClose) {
tlogFile.delete();
}
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
public String toString() {
return tlogFile.toString();
}
}
class ChannelFastInputStream extends FastInputStream {
FileChannel ch;
long chPosition;
public ChannelFastInputStream(FileChannel ch, long chPosition) {
super(null);
this.ch = ch;
this.chPosition = chPosition;
}
@Override
public int readWrappedStream(byte[] target, int offset, int len) throws IOException {
ByteBuffer bb = ByteBuffer.wrap(target, offset, len);
assert chPosition < ch.size();
for (;;) {
int ret = ch.read(bb, chPosition);
if (ret > 0) {
chPosition += ret;
return ret;
} else if (ret < 0) {
return ret;
}
// a channel read can return 0 - retry if this happens
}
}
@Override
public void close() throws IOException {
ch.close();
}
}

View File

@ -31,20 +31,25 @@ public class MergeIndexesCommand extends UpdateCommand {
public IndexReader[] readers;
public MergeIndexesCommand(IndexReader[] readers, SolrQueryRequest req) {
super("mergeIndexes", req);
super(req);
this.readers = readers;
}
@Override
public String name() {
return "mergeIndexes";
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(commandName);
sb.append(':');
StringBuilder sb = new StringBuilder(super.toString());
if (readers != null && readers.length > 0) {
sb.append(readers[0].directory());
for (int i = 1; i < readers.length; i++) {
sb.append(",").append(readers[i].directory());
}
}
sb.append('}');
return sb.toString();
}
}

View File

@ -0,0 +1,429 @@
/**
* 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.
*/
package org.apache.solr.update;
import java.io.IOException;
import java.net.ConnectException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.httpclient.NoHttpResponseException;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.SolrCore;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardHandlerFactory;
import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.handler.component.ShardResponse;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
import org.apache.solr.update.processor.RunUpdateProcessorFactory;
import org.apache.solr.update.processor.UpdateRequestProcessor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** @lucene.experimental */
public class PeerSync {
public static Logger log = LoggerFactory.getLogger(PeerSync.class);
public boolean debug = log.isDebugEnabled();
private List<String> replicas;
private int nUpdates;
private UpdateHandler uhandler;
private UpdateLog ulog;
private ShardHandlerFactory shardHandlerFactory;
private ShardHandler shardHandler;
private UpdateLog.RecentUpdates recentUpdates;
private List<Long> ourUpdates;
private Set<Long> ourUpdateSet;
private Set<Long> requestedUpdateSet;
private long ourLowThreshold; // 20th percentile
private long ourHighThreshold; // 80th percentile
// comparator that sorts by absolute value, putting highest first
private static Comparator<Long> absComparator = new Comparator<Long>() {
@Override
public int compare(Long o1, Long o2) {
long l1 = Math.abs(o1);
long l2 = Math.abs(o2);
if (l1 >l2) return -1;
if (l1 < l2) return 1;
return 0;
}
};
// comparator that sorts update records by absolute value of version, putting lowest first
private static Comparator<Object> updateRecordComparator = new Comparator<Object>() {
@Override
public int compare(Object o1, Object o2) {
if (!(o1 instanceof List)) return 1;
if (!(o2 instanceof List)) return -1;
List lst1 = (List)o1;
List lst2 = (List)o2;
long l1 = Math.abs((Long)lst1.get(1));
long l2 = Math.abs((Long)lst2.get(1));
if (l1 >l2) return 1;
if (l1 < l2) return -1;
return 0;
}
};
private static class SyncShardRequest extends ShardRequest {
List<Long> reportedVersions;
List<Long> requestedUpdates;
Exception updateException;
}
public PeerSync(SolrCore core, List<String> replicas, int nUpdates) {
this.replicas = replicas;
this.nUpdates = nUpdates;
uhandler = core.getUpdateHandler();
ulog = uhandler.getUpdateLog();
shardHandlerFactory = core.getCoreDescriptor().getCoreContainer().getShardHandlerFactory();
shardHandler = shardHandlerFactory.getShardHandler();
}
public long percentile(List<Long> arr, float frac) {
int elem = (int) (arr.size() * frac);
return Math.abs(arr.get(elem));
}
/** Returns true if peer sync was successful, meaning that this core may not be considered to have the latest updates.
* A commit is not performed.
*/
public boolean sync() {
if (ulog == null) {
return false;
}
// fire off the requests before getting our own recent updates (for better concurrency)
for (String replica : replicas) {
requestVersions(replica);
}
recentUpdates = ulog.getRecentUpdates();
try {
ourUpdates = recentUpdates.getVersions(nUpdates);
} finally {
recentUpdates.close();
}
Collections.sort(ourUpdates, absComparator);
if (ourUpdates.size() > 0) {
ourLowThreshold = percentile(ourUpdates, 0.8f);
ourHighThreshold = percentile(ourUpdates, 0.2f);
} else {
// we have no versions and hence no frame of reference to tell if we can use a peers
// updates to bring us into sync
return false;
}
ourUpdateSet = new HashSet<Long>(ourUpdates);
requestedUpdateSet = new HashSet<Long>(ourUpdates);
for(;;) {
ShardResponse srsp = shardHandler.takeCompletedOrError();
if (srsp == null) break;
boolean success = handleResponse(srsp);
if (!success) {
shardHandler.cancelAll();
return false;
}
}
return true;
}
private void requestVersions(String replica) {
SyncShardRequest sreq = new SyncShardRequest();
sreq.purpose = 1;
// TODO: this sucks
if (replica.startsWith("http://"))
replica = replica.substring(7);
sreq.shards = new String[]{replica};
sreq.actualShards = sreq.shards;
sreq.params = new ModifiableSolrParams();
sreq.params.set("qt","/get");
sreq.params.set("distrib",false);
sreq.params.set("getVersions",nUpdates);
shardHandler.submit(sreq, replica, sreq.params);
}
private boolean handleResponse(ShardResponse srsp) {
if (srsp.getException() != null) {
// TODO: look at this more thoroughly - we don't want
// to fail on connection exceptions, but it may make sense
// to determine this based on the number of fails
if (srsp.getException() instanceof SolrServerException) {
Throwable solrException = ((SolrServerException) srsp.getException())
.getRootCause();
if (solrException instanceof ConnectException
|| solrException instanceof NoHttpResponseException) {
return true;
}
}
// TODO: at least log???
// srsp.getException().printStackTrace(System.out);
return false;
}
ShardRequest sreq = srsp.getShardRequest();
if (sreq.purpose == 1) {
return handleVersions(srsp);
} else {
return handleUpdates(srsp);
}
}
private boolean handleVersions(ShardResponse srsp) {
// we retrieved the last N updates from the replica
List<Long> otherVersions = (List<Long>)srsp.getSolrResponse().getResponse().get("versions");
// TODO: how to handle short lists?
SyncShardRequest sreq = (SyncShardRequest) srsp.getShardRequest();
sreq.reportedVersions = otherVersions;
if (otherVersions.size() == 0) {
return true;
}
Collections.sort(otherVersions, absComparator);
long otherHigh = percentile(otherVersions, .2f);
long otherLow = percentile(otherVersions, .8f);
if (ourHighThreshold < otherLow) {
// Small overlap between version windows and ours is older
// This means that we might miss updates if we attempted to use this method.
// Since there exists just one replica that is so much newer, we must
// fail the sync.
return false;
}
if (ourLowThreshold > otherHigh) {
// Small overlap between windows and ours is newer.
// Using this list to sync would result in requesting/replaying results we don't need
// and possibly bringing deleted docs back to life.
return true;
}
List<Long> toRequest = new ArrayList<Long>();
for (Long otherVersion : otherVersions) {
// stop when the entries get old enough that reorders may lead us to see updates we don't need
if (Math.abs(otherVersion) < ourLowThreshold) break;
if (ourUpdateSet.contains(otherVersion) || requestedUpdateSet.contains(otherVersion)) {
// we either have this update, or already requested it
continue;
}
toRequest.add(otherVersion);
requestedUpdateSet.add(otherVersion);
}
sreq.requestedUpdates = toRequest;
if (toRequest.isEmpty()) {
// we had (or already requested) all the updates referenced by the replica
return true;
}
return requestUpdates(srsp, toRequest);
}
private boolean requestUpdates(ShardResponse srsp, List<Long> toRequest) {
String replica = srsp.getShardRequest().shards[0];
log.info("Requesting updates from " + replica + " versions=" + toRequest);
// reuse our original request object
ShardRequest sreq = srsp.getShardRequest();
sreq.purpose = 0;
sreq.params = new ModifiableSolrParams();
sreq.params.set("qt","/get");
sreq.params.set("distrib",false);
sreq.params.set("getUpdates", StrUtils.join(toRequest, ','));
sreq.responses.clear(); // needs to be zeroed for correct correlation to occur
shardHandler.submit(sreq, sreq.shards[0], sreq.params);
return true;
}
private boolean handleUpdates(ShardResponse srsp) {
// we retrieved the last N updates from the replica
List<Object> updates = (List<Object>)srsp.getSolrResponse().getResponse().get("updates");
SyncShardRequest sreq = (SyncShardRequest) srsp.getShardRequest();
if (updates.size() < sreq.requestedUpdates.size()) {
log.error("PeerSync: Requested " + sreq.requestedUpdates.size() + " updates from " + sreq.shards[0] + " but retrieved " + updates.size());
return false;
}
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(DistributedUpdateProcessor.SEEN_LEADER, true);
SolrQueryRequest req = new LocalSolrQueryRequest(uhandler.core, params);
SolrQueryResponse rsp = new SolrQueryResponse();
RunUpdateProcessorFactory runFac = new RunUpdateProcessorFactory();
DistributedUpdateProcessorFactory magicFac = new DistributedUpdateProcessorFactory();
runFac.init(new NamedList());
magicFac.init(new NamedList());
UpdateRequestProcessor proc = magicFac.getInstance(req, rsp, runFac.getInstance(req, rsp, null));
Collections.sort(updates, updateRecordComparator);
Object o = null;
long lastVersion = 0;
try {
// Apply oldest updates first
for (Object obj : updates) {
// should currently be a List<Oper,Ver,Doc/Id>
o = obj;
List<Object> entry = (List<Object>)o;
int oper = (Integer)entry.get(0);
long version = (Long) entry.get(1);
if (version == lastVersion && version != 0) continue;
lastVersion = version;
switch (oper) {
case UpdateLog.ADD:
{
// byte[] idBytes = (byte[]) entry.get(2);
SolrInputDocument sdoc = (SolrInputDocument)entry.get(entry.size()-1);
AddUpdateCommand cmd = new AddUpdateCommand(req);
// cmd.setIndexedId(new BytesRef(idBytes));
cmd.solrDoc = sdoc;
cmd.setVersion(version);
cmd.setFlags(UpdateCommand.PEER_SYNC | UpdateCommand.IGNORE_AUTOCOMMIT);
proc.processAdd(cmd);
break;
}
case UpdateLog.DELETE:
{
byte[] idBytes = (byte[]) entry.get(2);
DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
cmd.setIndexedId(new BytesRef(idBytes));
cmd.setVersion(version);
cmd.setFlags(UpdateCommand.PEER_SYNC | UpdateCommand.IGNORE_AUTOCOMMIT);
proc.processDelete(cmd);
break;
}
case UpdateLog.DELETE_BY_QUERY:
{
String query = (String)entry.get(2);
DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
cmd.query = query;
cmd.setVersion(version);
cmd.setFlags(UpdateCommand.PEER_SYNC | UpdateCommand.IGNORE_AUTOCOMMIT);
proc.processDelete(cmd);
break;
}
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown Operation! " + oper);
}
}
}
catch (IOException e) {
// TODO: should this be handled separately as a problem with us?
// I guess it probably already will by causing replication to be kicked off.
sreq.updateException = e;
log.error("Error applying updates from " + sreq.shards + " ,update=" + o, e);
return false;
}
catch (Exception e) {
sreq.updateException = e;
log.error("Error applying updates from " + sreq.shards + " ,update=" + o, e);
return false;
}
finally {
try {
proc.finish();
} catch (Exception e) {
sreq.updateException = e;
log.error("Error applying updates from " + sreq.shards + " ,finish()", e);
return false;
}
}
return true;
}
/** Requests and applies recent updates from peers */
public static void sync(SolrCore core, List<String> replicas, int nUpdates) {
UpdateHandler uhandler = core.getUpdateHandler();
ShardHandlerFactory shardHandlerFactory = core.getCoreDescriptor().getCoreContainer().getShardHandlerFactory();
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
for (String replica : replicas) {
ShardRequest sreq = new ShardRequest();
sreq.shards = new String[]{replica};
sreq.params = new ModifiableSolrParams();
sreq.params.set("qt","/get");
sreq.params.set("distrib", false);
sreq.params.set("getVersions",nUpdates);
shardHandler.submit(sreq, replica, sreq.params);
}
for (String replica : replicas) {
ShardResponse srsp = shardHandler.takeCompletedOrError();
}
}
}

View File

@ -26,7 +26,16 @@ import org.apache.solr.request.SolrQueryRequest;
public class RollbackUpdateCommand extends UpdateCommand {
public RollbackUpdateCommand(SolrQueryRequest req) {
super("rollback", req);
super(req);
}
@Override
public String name() {
return "rollback";
}
@Override
public String toString() {
return super.toString() + '}';
}
}

View File

@ -0,0 +1,484 @@
package org.apache.solr.update;
/**
* 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.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.httpclient.HttpClient;
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.UpdateRequestExt;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.SolrCore;
import org.apache.solr.util.DefaultSolrThreadFactory;
public class SolrCmdDistributor {
// TODO: shut this thing down
// TODO: this cannot be per instance...
static ThreadPoolExecutor commExecutor = new ThreadPoolExecutor(0,
Integer.MAX_VALUE, 5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
new DefaultSolrThreadFactory("cmdDistribExecutor"));
static HttpClient client;
static {
MultiThreadedHttpConnectionManager mgr = new MultiThreadedHttpConnectionManager();
mgr.getParams().setDefaultMaxConnectionsPerHost(8);
mgr.getParams().setMaxTotalConnections(200);
client = new HttpClient(mgr);
}
CompletionService<Request> completionService;
Set<Future<Request>> pending;
int maxBufferedAddsPerServer = 10;
int maxBufferedDeletesPerServer = 10;
private Response response = new Response();
private final Map<Node,List<AddRequest>> adds = new HashMap<Node,List<AddRequest>>();
private final Map<Node,List<DeleteRequest>> deletes = new HashMap<Node,List<DeleteRequest>>();
class AddRequest {
AddUpdateCommand cmd;
ModifiableSolrParams params;
}
class DeleteRequest {
DeleteUpdateCommand cmd;
ModifiableSolrParams params;
}
public SolrCmdDistributor() {
}
public void finish() {
// piggyback on any outstanding adds or deletes if possible.
flushAdds(1, null, null);
flushDeletes(1, null, null);
checkResponses(true);
}
public void distribDelete(DeleteUpdateCommand cmd, List<Node> urls, ModifiableSolrParams params) throws IOException {
checkResponses(false);
if (cmd.isDeleteById()) {
doDelete(cmd, urls, params);
} else {
doDelete(cmd, urls, params);
}
}
public void distribAdd(AddUpdateCommand cmd, List<Node> nodes, ModifiableSolrParams commitParams) throws IOException {
checkResponses(false);
// make sure any pending deletes are flushed
flushDeletes(1, null, null);
// TODO: this is brittle
// need to make a clone since these commands may be reused
AddUpdateCommand clone = new AddUpdateCommand(null);
clone.solrDoc = cmd.solrDoc;
clone.commitWithin = cmd.commitWithin;
clone.overwrite = cmd.overwrite;
clone.setVersion(cmd.getVersion());
AddRequest addRequest = new AddRequest();
addRequest.cmd = clone;
addRequest.params = commitParams;
for (Node node : nodes) {
List<AddRequest> alist = adds.get(node);
if (alist == null) {
alist = new ArrayList<AddRequest>(2);
adds.put(node, alist);
}
alist.add(addRequest);
}
flushAdds(maxBufferedAddsPerServer, null, null);
}
public void distribCommit(CommitUpdateCommand cmd, List<Node> nodes,
ModifiableSolrParams params) throws IOException {
// Wait for all outstanding responses to make sure that a commit
// can't sneak in ahead of adds or deletes we already sent.
// We could do this on a per-server basis, but it's more complex
// and this solution will lead to commits happening closer together.
checkResponses(true);
// currently, we dont try to piggy back on outstanding adds or deletes
UpdateRequestExt ureq = new UpdateRequestExt();
ureq.setParams(params);
addCommit(ureq, cmd);
for (Node node : nodes) {
submit(ureq, node);
}
// if the command wanted to block until everything was committed,
// then do that here.
if (cmd.waitSearcher) {
checkResponses(true);
}
}
private void doDelete(DeleteUpdateCommand cmd, List<Node> nodes,
ModifiableSolrParams params) throws IOException {
flushAdds(1, null, null);
DeleteUpdateCommand clonedCmd = clone(cmd);
DeleteRequest deleteRequest = new DeleteRequest();
deleteRequest.cmd = clonedCmd;
deleteRequest.params = params;
for (Node node : nodes) {
List<DeleteRequest> dlist = deletes.get(node);
if (dlist == null) {
dlist = new ArrayList<DeleteRequest>(2);
deletes.put(node, dlist);
}
dlist.add(deleteRequest);
}
flushDeletes(maxBufferedDeletesPerServer, null, null);
}
void addCommit(UpdateRequestExt ureq, CommitUpdateCommand cmd) {
if (cmd == null) return;
ureq.setAction(cmd.optimize ? AbstractUpdateRequest.ACTION.OPTIMIZE
: AbstractUpdateRequest.ACTION.COMMIT, false, cmd.waitSearcher);
}
boolean flushAdds(int limit, CommitUpdateCommand ccmd, ModifiableSolrParams commitParams) {
// check for pending deletes
Set<Node> removeNodes = new HashSet<Node>();
Set<Node> nodes = adds.keySet();
for (Node node : nodes) {
List<AddRequest> alist = adds.get(node);
if (alist == null || alist.size() < limit) return false;
UpdateRequestExt ureq = new UpdateRequestExt();
addCommit(ureq, ccmd);
ModifiableSolrParams combinedParams = new ModifiableSolrParams();
for (AddRequest aReq : alist) {
AddUpdateCommand cmd = aReq.cmd;
combinedParams.add(aReq.params);
ureq.add(cmd.solrDoc, cmd.commitWithin, cmd.overwrite);
}
if (commitParams != null) combinedParams.add(commitParams);
if (ureq.getParams() == null) ureq.setParams(new ModifiableSolrParams());
ureq.getParams().add(combinedParams);
removeNodes.add(node);
submit(ureq, node);
}
for (Node node : removeNodes) {
adds.remove(node);
}
return true;
}
boolean flushDeletes(int limit, CommitUpdateCommand ccmd, ModifiableSolrParams commitParams) {
// check for pending deletes
Set<Node> removeNodes = new HashSet<Node>();
Set<Node> nodes = deletes.keySet();
for (Node node : nodes) {
List<DeleteRequest> dlist = deletes.get(node);
if (dlist == null || dlist.size() < limit) return false;
UpdateRequestExt ureq = new UpdateRequestExt();
addCommit(ureq, ccmd);
ModifiableSolrParams combinedParams = new ModifiableSolrParams();
for (DeleteRequest dReq : dlist) {
DeleteUpdateCommand cmd = dReq.cmd;
combinedParams.add(dReq.params);
if (cmd.isDeleteById()) {
ureq.deleteById(cmd.getId(), cmd.getVersion());
} else {
ureq.deleteByQuery(cmd.query);
}
if (commitParams != null) combinedParams.add(commitParams);
if (ureq.getParams() == null) ureq
.setParams(new ModifiableSolrParams());
ureq.getParams().add(combinedParams);
}
removeNodes.add(node);
submit(ureq, node);
}
for (Node node : removeNodes) {
deletes.remove(node);
}
return true;
}
private DeleteUpdateCommand clone(DeleteUpdateCommand cmd) {
DeleteUpdateCommand c = (DeleteUpdateCommand)cmd.clone();
// TODO: shouldnt the clone do this?
c.setFlags(cmd.getFlags());
c.setVersion(cmd.getVersion());
return c;
}
public static class Request {
public Node node;
UpdateRequestExt ureq;
NamedList<Object> ursp;
int rspCode;
public Exception exception;
int retries;
}
void submit(UpdateRequestExt ureq, Node node) {
Request sreq = new Request();
sreq.node = node;
sreq.ureq = ureq;
submit(sreq);
}
public void submit(final Request sreq) {
if (completionService == null) {
completionService = new ExecutorCompletionService<Request>(commExecutor);
pending = new HashSet<Future<Request>>();
}
final String url = sreq.node.getUrl();
Callable<Request> task = new Callable<Request>() {
@Override
public Request call() throws Exception {
Request clonedRequest = new Request();
clonedRequest.node = sreq.node;
clonedRequest.ureq = sreq.ureq;
clonedRequest.retries = sreq.retries;
try {
String fullUrl;
if (!url.startsWith("http://") && !url.startsWith("https://")) {
fullUrl = "http://" + url;
} else {
fullUrl = url;
}
CommonsHttpSolrServer server = new CommonsHttpSolrServer(fullUrl,
client);
clonedRequest.ursp = server.request(clonedRequest.ureq);
// currently no way to get the request body.
} catch (Exception e) {
clonedRequest.exception = e;
if (e instanceof SolrException) {
clonedRequest.rspCode = ((SolrException) e).code();
} else {
clonedRequest.rspCode = -1;
}
}
return clonedRequest;
}
};
pending.add(completionService.submit(task));
}
void checkResponses(boolean block) {
while (pending != null && pending.size() > 0) {
try {
Future<Request> future = block ? completionService.take()
: completionService.poll();
if (future == null) return;
pending.remove(future);
try {
Request sreq = future.get();
if (sreq.rspCode != 0) {
// error during request
// if there is a retry url, we want to retry...
// TODO: but we really should only retry on connection errors...
if (sreq.retries < 5 && sreq.node.checkRetry()) {
sreq.retries++;
sreq.rspCode = 0;
sreq.exception = null;
Thread.sleep(500);
submit(sreq);
checkResponses(block);
} else {
Exception e = sreq.exception;
Error error = new Error();
error.e = e;
error.node = sreq.node;
response.errors.add(error);
response.sreq = sreq;
SolrException.log(SolrCore.log, "shard update error "
+ sreq.node, sreq.exception);
}
}
} catch (ExecutionException e) {
// shouldn't happen since we catch exceptions ourselves
SolrException.log(SolrCore.log,
"error sending update request to shard", e);
}
} catch (InterruptedException e) {
throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,
"interrupted waiting for shard update response", e);
}
}
}
public static class Response {
public Request sreq;
public List<Error> errors = new ArrayList<Error>();
}
public static class Error {
public Node node;
public Exception e;
}
public Response getResponse() {
return response;
}
public static abstract class Node {
public abstract String getUrl();
public abstract boolean checkRetry();
public abstract String getCoreName();
public abstract String getBaseUrl();
public abstract ZkCoreNodeProps getNodeProps();
}
public static class StdNode extends Node {
protected String url;
protected String baseUrl;
protected String coreName;
private ZkCoreNodeProps nodeProps;
public StdNode(ZkCoreNodeProps nodeProps) {
this.url = nodeProps.getCoreUrl();
this.baseUrl = nodeProps.getBaseUrl();
this.coreName = nodeProps.getCoreName();
this.nodeProps = nodeProps;
}
@Override
public String getUrl() {
return url;
}
@Override
public String toString() {
return this.getClass().getSimpleName() + ": " + url;
}
@Override
public boolean checkRetry() {
return false;
}
@Override
public String getBaseUrl() {
return baseUrl;
}
@Override
public String getCoreName() {
return coreName;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + ((baseUrl == null) ? 0 : baseUrl.hashCode());
result = prime * result + ((coreName == null) ? 0 : coreName.hashCode());
result = prime * result + ((url == null) ? 0 : url.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (obj == null) return false;
if (getClass() != obj.getClass()) return false;
StdNode other = (StdNode) obj;
if (baseUrl == null) {
if (other.baseUrl != null) return false;
} else if (!baseUrl.equals(other.baseUrl)) return false;
if (coreName == null) {
if (other.coreName != null) return false;
} else if (!coreName.equals(other.coreName)) return false;
if (url == null) {
if (other.url != null) return false;
} else if (!url.equals(other.url)) return false;
return true;
}
public ZkCoreNodeProps getNodeProps() {
return nodeProps;
}
}
}

View File

@ -49,11 +49,12 @@ public abstract class SolrCoreState {
/**
* Decrement the number of references to this state. When then number of
* references hits 0, the state will close.
* references hits 0, the state will close. If an optional closer is
* passed, that will be used to close the writer.
*
* @throws IOException
*/
public abstract void decref() throws IOException;
public abstract void decref(IndexWriterCloser closer) throws IOException;
/**
* Increment the number of references to this state.
@ -74,4 +75,13 @@ public abstract class SolrCoreState {
*/
public abstract DirectoryFactory getDirectoryFactory();
public interface IndexWriterCloser {
public void closeWriter(IndexWriter writer) throws IOException;
}
public abstract void doRecovery(SolrCore core);
public abstract void cancelRecovery();
}

View File

@ -0,0 +1,738 @@
/**
* 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.
*/
package org.apache.solr.update;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.util.FastInputStream;
import org.apache.solr.common.util.FastOutputStream;
import org.apache.solr.common.util.JavaBinCodec;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.*;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.FileChannel;
import java.rmi.registry.LocateRegistry;
import java.util.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Log Format: List{Operation, Version, ...}
* ADD, VERSION, DOC
* DELETE, VERSION, ID_BYTES
* DELETE_BY_QUERY, VERSION, String
*
* TODO: keep two files, one for [operation, version, id] and the other for the actual
* document data. That way we could throw away document log files more readily
* while retaining the smaller operation log files longer (and we can retrieve
* the stored fields from the latest documents from the index).
*
* This would require keeping all source fields stored of course.
*
* This would also allow to not log document data for requests with commit=true
* in them (since we know that if the request succeeds, all docs will be committed)
*
*/
public class TransactionLog {
public static Logger log = LoggerFactory.getLogger(TransactionLog.class);
public final static String END_MESSAGE="SOLR_TLOG_END";
long id;
File tlogFile;
RandomAccessFile raf;
FileChannel channel;
OutputStream os;
FastOutputStream fos; // all accesses to this stream should be synchronized on "this" (The TransactionLog)
int numRecords;
volatile boolean deleteOnClose = true; // we can delete old tlogs since they are currently only used for real-time-get (and in the future, recovery)
AtomicInteger refcount = new AtomicInteger(1);
Map<String,Integer> globalStringMap = new HashMap<String, Integer>();
List<String> globalStringList = new ArrayList<String>();
final boolean debug = log.isDebugEnabled();
long snapshot_size;
int snapshot_numRecords;
// write a BytesRef as a byte array
JavaBinCodec.ObjectResolver resolver = new JavaBinCodec.ObjectResolver() {
@Override
public Object resolve(Object o, JavaBinCodec codec) throws IOException {
if (o instanceof BytesRef) {
BytesRef br = (BytesRef)o;
codec.writeByteArray(br.bytes, br.offset, br.length);
return null;
}
return o;
}
};
public class LogCodec extends JavaBinCodec {
public LogCodec() {
super(resolver);
}
@Override
public void writeExternString(String s) throws IOException {
if (s == null) {
writeTag(NULL);
return;
}
// no need to synchronize globalStringMap - it's only updated before the first record is written to the log
Integer idx = globalStringMap.get(s);
if (idx == null) {
// write a normal string
writeStr(s);
} else {
// write the extern string
writeTag(EXTERN_STRING, idx);
}
}
@Override
public String readExternString(FastInputStream fis) throws IOException {
int idx = readSize(fis);
if (idx != 0) {// idx != 0 is the index of the extern string
// no need to synchronize globalStringList - it's only updated before the first record is written to the log
return globalStringList.get(idx - 1);
} else {// idx == 0 means it has a string value
// this shouldn't happen with this codec subclass.
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Corrupt transaction log");
}
}
}
TransactionLog(File tlogFile, Collection<String> globalStrings) throws IOException {
this(tlogFile, globalStrings, false);
}
TransactionLog(File tlogFile, Collection<String> globalStrings, boolean openExisting) throws IOException {
try {
if (debug) {
log.debug("New TransactionLog file=" + tlogFile + ", exists=" + tlogFile.exists() + ", size=" + tlogFile.length() + ", openExisting=" + openExisting);
}
this.tlogFile = tlogFile;
raf = new RandomAccessFile(this.tlogFile, "rw");
long start = raf.length();
channel = raf.getChannel();
os = Channels.newOutputStream(channel);
fos = FastOutputStream.wrap(os);
if (openExisting) {
if (start > 0) {
readHeader(null);
raf.seek(start);
assert channel.position() == start;
fos.setWritten(start); // reflect that we aren't starting at the beginning
assert fos.size() == channel.size();
} else {
addGlobalStrings(globalStrings);
}
} else {
assert start==0;
if (start > 0) {
raf.setLength(0);
}
addGlobalStrings(globalStrings);
}
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
/** Returns the number of records in the log (currently includes the header and an optional commit).
* Note: currently returns 0 for reopened existing log files.
*/
public int numRecords() {
synchronized (this) {
return this.numRecords;
}
}
public boolean endsWithCommit() throws IOException {
long size;
synchronized (this) {
fos.flush();
size = fos.size();
}
// the end of the file should have the end message (added during a commit) plus a 4 byte size
byte[] buf = new byte[ END_MESSAGE.length() ];
long pos = size - END_MESSAGE.length() - 4;
if (pos < 0) return false;
ChannelFastInputStream is = new ChannelFastInputStream(channel, pos);
is.read(buf);
for (int i=0; i<buf.length; i++) {
if (buf[i] != END_MESSAGE.charAt(i)) return false;
}
return true;
}
/** takes a snapshot of the current position and number of records
* for later possible rollback, and returns the position */
public long snapshot() {
synchronized (this) {
snapshot_size = fos.size();
snapshot_numRecords = numRecords;
return snapshot_size;
}
}
// This could mess with any readers or reverse readers that are open, or anything that might try to do a log lookup.
// This should only be used to roll back buffered updates, not actually applied updates.
public void rollback(long pos) throws IOException {
synchronized (this) {
assert snapshot_size == pos;
fos.flush();
raf.setLength(pos);
fos.setWritten(pos);
assert fos.size() == pos;
numRecords = snapshot_numRecords;
}
}
public long writeData(Object o) {
LogCodec codec = new LogCodec();
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
codec.init(fos);
codec.writeVal(o);
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
private void readHeader(FastInputStream fis) throws IOException {
// read existing header
fis = fis != null ? fis : new ChannelFastInputStream(channel, 0);
LogCodec codec = new LogCodec();
Map header = (Map)codec.unmarshal(fis);
fis.readInt(); // skip size
// needed to read other records
synchronized (this) {
globalStringList = (List<String>)header.get("strings");
globalStringMap = new HashMap<String, Integer>(globalStringList.size());
for (int i=0; i<globalStringList.size(); i++) {
globalStringMap.put( globalStringList.get(i), i+1);
}
}
}
private void addGlobalStrings(Collection<String> strings) {
if (strings == null) return;
int origSize = globalStringMap.size();
for (String s : strings) {
Integer idx = null;
if (origSize > 0) {
idx = globalStringMap.get(s);
}
if (idx != null) continue; // already in list
globalStringList.add(s);
globalStringMap.put(s, globalStringList.size());
}
assert globalStringMap.size() == globalStringList.size();
}
Collection<String> getGlobalStrings() {
synchronized (this) {
return new ArrayList<String>(globalStringList);
}
}
private void writeLogHeader(LogCodec codec) throws IOException {
long pos = fos.size();
assert pos == 0;
Map header = new LinkedHashMap<String,Object>();
header.put("SOLR_TLOG",1); // a magic string + version number
header.put("strings",globalStringList);
codec.marshal(header, fos);
endRecord(pos);
}
private void endRecord(long startRecordPosition) throws IOException {
fos.writeInt((int)(fos.size() - startRecordPosition));
numRecords++;
}
public long write(AddUpdateCommand cmd) {
LogCodec codec = new LogCodec();
long pos = 0;
synchronized (this) {
try {
pos = fos.size(); // if we had flushed, this should be equal to channel.position()
SolrInputDocument sdoc = cmd.getSolrInputDocument();
if (pos == 0) { // TODO: needs to be changed if we start writing a header first
addGlobalStrings(sdoc.getFieldNames());
writeLogHeader(codec);
pos = fos.size();
}
/***
System.out.println("###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
if (pos != fos.size()) {
throw new RuntimeException("ERROR" + "###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
}
***/
codec.init(fos);
codec.writeTag(JavaBinCodec.ARR, 3);
codec.writeInt(UpdateLog.ADD); // should just take one byte
codec.writeLong(cmd.getVersion());
codec.writeSolrInputDocument(cmd.getSolrInputDocument());
endRecord(pos);
// fos.flushBuffer(); // flush later
return pos;
} catch (IOException e) {
// TODO: reset our file pointer back to "pos", the start of this record.
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error logging add", e);
}
}
}
public long writeDelete(DeleteUpdateCommand cmd) {
LogCodec codec = new LogCodec();
synchronized (this) {
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
if (pos == 0) {
writeLogHeader(codec);
pos = fos.size();
}
codec.init(fos);
codec.writeTag(JavaBinCodec.ARR, 3);
codec.writeInt(UpdateLog.DELETE); // should just take one byte
codec.writeLong(cmd.getVersion());
BytesRef br = cmd.getIndexedId();
codec.writeByteArray(br.bytes, br.offset, br.length);
endRecord(pos);
// fos.flushBuffer(); // flush later
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
}
public long writeDeleteByQuery(DeleteUpdateCommand cmd) {
LogCodec codec = new LogCodec();
synchronized (this) {
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
if (pos == 0) {
writeLogHeader(codec);
pos = fos.size();
}
codec.init(fos);
codec.writeTag(JavaBinCodec.ARR, 3);
codec.writeInt(UpdateLog.DELETE_BY_QUERY); // should just take one byte
codec.writeLong(cmd.getVersion());
codec.writeStr(cmd.query);
endRecord(pos);
// fos.flushBuffer(); // flush later
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
}
public long writeCommit(CommitUpdateCommand cmd) {
LogCodec codec = new LogCodec();
synchronized (this) {
try {
long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
if (pos == 0) {
writeLogHeader(codec);
pos = fos.size();
}
codec.init(fos);
codec.writeTag(JavaBinCodec.ARR, 3);
codec.writeInt(UpdateLog.COMMIT); // should just take one byte
codec.writeLong(cmd.getVersion());
codec.writeStr(END_MESSAGE); // ensure these bytes are (almost) last in the file
endRecord(pos);
fos.flush(); // flush since this will be the last record in a log fill
assert fos.size() == channel.size();
return pos;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
}
/* This method is thread safe */
public Object lookup(long pos) {
// A negative position can result from a log replay (which does not re-log, but does
// update the version map. This is OK since the node won't be ACTIVE when this happens.
if (pos < 0) return null;
try {
// make sure any unflushed buffer has been flushed
synchronized (this) {
// TODO: optimize this by keeping track of what we have flushed up to
fos.flushBuffer();
/***
System.out.println("###flushBuffer to " + fos.size() + " raf.length()=" + raf.length() + " pos="+pos);
if (fos.size() != raf.length() || pos >= fos.size() ) {
throw new RuntimeException("ERROR" + "###flushBuffer to " + fos.size() + " raf.length()=" + raf.length() + " pos="+pos);
}
***/
}
ChannelFastInputStream fis = new ChannelFastInputStream(channel, pos);
LogCodec codec = new LogCodec();
return codec.readVal(fis);
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
public void incref() {
int result = refcount.incrementAndGet();
if (result <= 1) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "incref on a closed log: " + this);
}
}
public boolean try_incref() {
return refcount.incrementAndGet() > 1;
}
public void decref() {
if (refcount.decrementAndGet() == 0) {
close();
}
}
/** returns the current position in the log file */
public long position() {
synchronized (this) {
return fos.size();
}
}
public void finish(UpdateLog.SyncLevel syncLevel) {
if (syncLevel == UpdateLog.SyncLevel.NONE) return;
try {
synchronized (this) {
fos.flushBuffer();
}
if (syncLevel == UpdateLog.SyncLevel.FSYNC) {
// Since fsync is outside of synchronized block, we can end up with a partial
// last record on power failure (which is OK, and does not represent an error...
// we just need to be aware of it when reading).
raf.getFD().sync();
}
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
private void close() {
try {
if (debug) {
log.debug("Closing tlog" + this);
}
synchronized (this) {
fos.flush();
fos.close();
}
if (deleteOnClose) {
tlogFile.delete();
}
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
public void forceClose() {
if (refcount.get() > 0) {
log.error("Error: Forcing close of " + this);
refcount.set(0);
close();
}
}
public String toString() {
return "tlog{file=" + tlogFile.toString() + " refcount=" + refcount.get() + "}";
}
/** Returns a reader that can be used while a log is still in use.
* Currently only *one* LogReader may be outstanding, and that log may only
* be used from a single thread. */
public LogReader getReader(long startingPos) {
return new LogReader(startingPos);
}
/** Returns a single threaded reverse reader */
public ReverseReader getReverseReader() throws IOException {
return new ReverseReader();
}
public class LogReader {
ChannelFastInputStream fis;
private LogCodec codec = new LogCodec();
public LogReader(long startingPos) {
incref();
fis = new ChannelFastInputStream(channel, startingPos);
}
/** Returns the next object from the log, or null if none available.
*
* @return The log record, or null if EOF
* @throws IOException
*/
public Object next() throws IOException, InterruptedException {
long pos = fis.position();
synchronized (TransactionLog.this) {
if (debug) {
log.debug("Reading log record. pos="+pos+" currentSize="+fos.size());
}
if (pos >= fos.size()) {
return null;
}
fos.flushBuffer();
}
if (pos == 0) {
readHeader(fis);
// shouldn't currently happen - header and first record are currently written at the same time
synchronized (TransactionLog.this) {
if (fis.position() >= fos.size()) {
return null;
}
pos = fis.position();
}
}
Object o = codec.readVal(fis);
// skip over record size
int size = fis.readInt();
assert size == fis.position() - pos - 4;
return o;
}
public void close() {
decref();
}
@Override
public String toString() {
synchronized (TransactionLog.this) {
return "LogReader{" + "file=" + tlogFile + ", position=" + fis.position() + ", end=" + fos.size() + "}";
}
}
}
public class ReverseReader {
ChannelFastInputStream fis;
private LogCodec codec = new LogCodec() {
@Override
public SolrInputDocument readSolrInputDocument(FastInputStream dis) throws IOException {
// Given that the SolrInputDocument is last in an add record, it's OK to just skip
// reading it completely.
return null;
}
};
int nextLength; // length of the next record (the next one closer to the start of the log file)
long prevPos; // where we started reading from last time (so prevPos - nextLength == start of next record)
public ReverseReader() throws IOException {
incref();
long sz;
synchronized (TransactionLog.this) {
fos.flushBuffer();
sz = fos.size();
assert sz == channel.size();
}
fis = new ChannelFastInputStream(channel, 0);
if (sz >=4) {
// readHeader(fis); // should not be needed
prevPos = sz - 4;
fis.seek(prevPos);
nextLength = fis.readInt();
}
}
/** Returns the next object from the log, or null if none available.
*
* @return The log record, or null if EOF
* @throws IOException
*/
public Object next() throws IOException {
if (prevPos <= 0) return null;
long endOfThisRecord = prevPos;
int thisLength = nextLength;
long recordStart = prevPos - thisLength; // back up to the beginning of the next record
prevPos = recordStart - 4; // back up 4 more to read the length of the next record
if (prevPos <= 0) return null; // this record is the header
long bufferPos = fis.getBufferPos();
if (prevPos >= bufferPos) {
// nothing to do... we're within the current buffer
} else {
// Position buffer so that this record is at the end.
// For small records, this will cause subsequent calls to next() to be within the buffer.
long seekPos = endOfThisRecord - fis.getBufferSize();
seekPos = Math.min(seekPos, prevPos); // seek to the start of the record if it's larger then the block size.
seekPos = Math.max(seekPos, 0);
fis.seek(seekPos);
fis.peek(); // cause buffer to be filled
}
fis.seek(prevPos);
nextLength = fis.readInt(); // this is the length of the *next* record (i.e. closer to the beginning)
// TODO: optionally skip document data
Object o = codec.readVal(fis);
// assert fis.position() == prevPos + 4 + thisLength; // this is only true if we read all the data (and we currently skip reading SolrInputDocument
return o;
}
/* returns the position in the log file of the last record returned by next() */
public long position() {
return prevPos + 4; // skip the length
}
public void close() {
decref();
}
@Override
public String toString() {
synchronized (TransactionLog.this) {
return "LogReader{" + "file=" + tlogFile + ", position=" + fis.position() + ", end=" + fos.size() + "}";
}
}
}
}
class ChannelFastInputStream extends FastInputStream {
private FileChannel ch;
public ChannelFastInputStream(FileChannel ch, long chPosition) {
// super(null, new byte[10],0,0); // a small buffer size for testing purposes
super(null);
this.ch = ch;
super.readFromStream = chPosition;
}
@Override
public int readWrappedStream(byte[] target, int offset, int len) throws IOException {
ByteBuffer bb = ByteBuffer.wrap(target, offset, len);
int ret = ch.read(bb, readFromStream);
return ret;
}
public void seek(long position) throws IOException {
if (position <= readFromStream && position >= getBufferPos()) {
// seek within buffer
pos = (int)(position - getBufferPos());
} else {
// long currSize = ch.size(); // not needed - underlying read should handle (unless read never done)
// if (position > currSize) throw new EOFException("Read past EOF: seeking to " + position + " on file of size " + currSize + " file=" + ch);
readFromStream = position;
end = pos = 0;
}
assert position() == position;
}
/** where is the start of the buffer relative to the whole file */
public long getBufferPos() {
return readFromStream - end;
}
public int getBufferSize() {
return buf.length;
}
@Override
public void close() throws IOException {
ch.close();
}
@Override
public String toString() {
return "readFromStream="+readFromStream +" pos="+pos +" end="+end + " bufferPos="+getBufferPos() + " position="+position() ;
}
}

View File

@ -17,6 +17,7 @@
package org.apache.solr.update;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.request.SolrQueryRequest;
@ -24,17 +25,56 @@ import org.apache.solr.request.SolrQueryRequest;
*
*
*/
public class UpdateCommand {
protected final SolrQueryRequest req;
protected final String commandName;
public abstract class UpdateCommand implements Cloneable {
protected SolrQueryRequest req;
protected long version;
protected int flags;
public UpdateCommand(String commandName, SolrQueryRequest req) {
public static int BUFFERING = 0x00000001; // update command is being buffered.
public static int REPLAY = 0x00000002; // update command is from replaying a log.
public static int PEER_SYNC = 0x00000004; // update command is a missing update being provided by a peer.
public static int IGNORE_AUTOCOMMIT = 0x00000008; // this update should not count toward triggering of autocommits.
public UpdateCommand(SolrQueryRequest req) {
this.req = req;
this.commandName = commandName;
}
public abstract String name();
@Override
public String toString() {
return commandName;
return name() + "{flags="+flags+",version="+version;
}
public long getVersion() {
return version;
}
public void setVersion(long version) {
this.version = version;
}
public void setFlags(int flags) {
this.flags = flags;
}
public int getFlags() {
return flags;
}
public SolrQueryRequest getReq() {
return req;
}
public void setReq(SolrQueryRequest req) {
this.req = req;
}
@Override
public UpdateCommand clone() {
try {
return (UpdateCommand) super.clone();
} catch (CloneNotSupportedException e) {
return null;
}
}
}

View File

@ -88,13 +88,12 @@ public abstract class UpdateHandler implements SolrInfoMBean {
private void initLog() {
PluginInfo ulogPluginInfo = core.getSolrConfig().getPluginInfo(UpdateLog.class.getName());
if (ulogPluginInfo != null && ulogPluginInfo.isEnabled()) {
ulog = core.createInitInstance(ulogPluginInfo, UpdateLog.class, "update log", "solr.NullUpdateLog");
} else {
ulog = new NullUpdateLog();
ulog.init(null);
}
ulog = new UpdateLog();
ulog.init(ulogPluginInfo);
// ulog = core.createInitInstance(ulogPluginInfo, UpdateLog.class, "update log", "solr.NullUpdateLog");
ulog.init(this, core);
}
}
protected void callPostCommitCallbacks() {
@ -124,15 +123,6 @@ public abstract class UpdateHandler implements SolrInfoMBean {
initLog();
}
/**
* Allows the UpdateHandler to create the SolrIndexSearcher after it
* has issued a 'softCommit'.
*
* @param previousSearcher
* @throws IOException
*/
public abstract SolrIndexSearcher reopenSearcher(SolrIndexSearcher previousSearcher) throws IOException;
/**
* Called when the Writer should be opened again - eg when replication replaces
* all of the index files.
@ -141,7 +131,7 @@ public abstract class UpdateHandler implements SolrInfoMBean {
*/
public abstract void newIndexWriter() throws IOException;
public abstract SolrCoreState getIndexWriterProvider();
public abstract SolrCoreState getSolrCoreState();
public abstract int addDoc(AddUpdateCommand cmd) throws IOException;
public abstract void delete(DeleteUpdateCommand cmd) throws IOException;

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,33 @@
/**
* 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.
*/
package org.apache.solr.update;
// TODO: make inner?
// TODO: store the highest possible in the index on a commit (but how to not block adds?)
// TODO: could also store highest possible in the transaction log after a commit.
// Or on a new index, just scan "version" for the max?
/** @lucene.internal */
public class VersionBucket {
public long highest;
public void updateHighest(long val) {
if (highest != 0) {
highest = Math.max(highest, Math.abs(val));
}
}
}

View File

@ -0,0 +1,175 @@
/**
* 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.
*/
package org.apache.solr.update;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.common.SolrException;
import org.apache.solr.core.SolrCore;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.util.RefCounted;
public class VersionInfo {
public static final String VERSION_FIELD="_version_";
private SolrCore core;
private UpdateHandler updateHandler;
private final VersionBucket[] buckets;
private SchemaField versionField;
private SchemaField idField;
final ReadWriteLock lock = new ReentrantReadWriteLock(true);
public VersionInfo(UpdateHandler updateHandler, int nBuckets) {
this.updateHandler = updateHandler;
this.core = updateHandler.core;
versionField = core.getSchema().getFieldOrNull(VERSION_FIELD);
idField = core.getSchema().getUniqueKeyField();
buckets = new VersionBucket[ BitUtil.nextHighestPowerOfTwo(nBuckets) ];
for (int i=0; i<buckets.length; i++) {
buckets[i] = new VersionBucket();
}
}
public SchemaField getVersionField() {
return versionField;
}
public void lockForUpdate() {
lock.readLock().lock();
}
public void unlockForUpdate() {
lock.readLock().unlock();
}
public void blockUpdates() {
lock.writeLock().lock();
}
public void unblockUpdates() {
lock.writeLock().unlock();
}
/***
// todo: initialize... use current time to start?
// a clock that increments by 1 for every operation makes it easier to detect missing
// messages, but raises other issues:
// - need to initialize to largest thing in index or tlog
// - when becoming leader, need to make sure it's greater than
// - using to detect missing messages means we need to keep track per-leader, or make
// sure a new leader starts off with 1 greater than the last leader.
private final AtomicLong clock = new AtomicLong();
public long getNewClock() {
return clock.incrementAndGet();
}
// Named *old* to prevent accidental calling getClock and expecting a new updated clock.
public long getOldClock() {
return clock.get();
}
***/
/** We are currently using this time-based clock to avoid going back in time on a
* server restart (i.e. we don't want version numbers to start at 1 again).
*/
// Time-based lamport clock. Good for introducing some reality into clocks (to the degree
// that times are somewhat synchronized in the cluster).
// Good if we want to relax some constraints to scale down to where only one node may be
// up at a time. Possibly harder to detect missing messages (because versions are not contiguous.
long vclock;
long time;
private final Object clockSync = new Object();
public long getNewClock() {
synchronized (clockSync) {
time = System.currentTimeMillis();
long result = time << 20;
if (result <= vclock) {
result = vclock + 1;
}
vclock = result;
return vclock;
}
}
public long getOldClock() {
synchronized (clockSync) {
return vclock;
}
}
public void updateClock(long clock) {
synchronized (clockSync) {
vclock = Math.max(vclock, clock);
}
}
public VersionBucket bucket(int hash) {
// If this is a user provided hash, it may be poor in the right-hand bits.
// Make sure high bits are moved down, since only the low bits will matter.
// int h = hash + (hash >>> 8) + (hash >>> 16) + (hash >>> 24);
// Assume good hash codes for now.
int slot = hash & (buckets.length-1);
return buckets[slot];
}
public Long lookupVersion(BytesRef idBytes) {
return updateHandler.ulog.lookupVersion(idBytes);
}
public Long getVersionFromIndex(BytesRef idBytes) {
// TODO: we could cache much of this and invalidate during a commit.
// TODO: most DocValues classes are threadsafe - expose which.
RefCounted<SolrIndexSearcher> newestSearcher = core.getRealtimeSearcher();
try {
SolrIndexSearcher searcher = newestSearcher.get();
long lookup = searcher.lookupId(idBytes);
if (lookup < 0) return null;
ValueSource vs = versionField.getType().getValueSource(versionField, null);
Map context = ValueSource.newContext(searcher);
vs.createWeight(context, searcher);
FunctionValues fv = vs.getValues(context, searcher.getTopReaderContext().leaves()[(int)(lookup>>32)]);
long ver = fv.longVal((int)lookup);
return ver;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading version from index", e);
} finally {
if (newestSearcher != null) {
newestSearcher.decref();
}
}
}
}

View File

@ -0,0 +1,809 @@
package org.apache.solr.update.processor;
/**
* 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.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrInputField;
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.cloud.ZooKeeperException;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.Hash;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.update.AddUpdateCommand;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.DeleteUpdateCommand;
import org.apache.solr.update.SolrCmdDistributor;
import org.apache.solr.update.SolrCmdDistributor.Node;
import org.apache.solr.update.SolrCmdDistributor.Response;
import org.apache.solr.update.SolrCmdDistributor.StdNode;
import org.apache.solr.update.UpdateCommand;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.VersionBucket;
import org.apache.solr.update.VersionInfo;
// NOT mt-safe... create a new processor for each add thread
// TODO: we really should not wait for distrib after local? unless a certain replication factor is asked for
public class DistributedUpdateProcessor extends UpdateRequestProcessor {
public static final String SEEN_LEADER = "leader";
public static final String COMMIT_END_POINT = "commit_end_point";
public static final String DELQUERY_END_POINT = "delquery_end_point";
private final SolrQueryRequest req;
private final SolrQueryResponse rsp;
private final UpdateRequestProcessor next;
public static final String VERSION_FIELD = "_version_";
private final UpdateHandler updateHandler;
private final UpdateLog ulog;
private final VersionInfo vinfo;
private final boolean versionsStored;
private boolean returnVersions = true; // todo: default to false and make configurable
private NamedList addsResponse = null;
private NamedList deleteResponse = null;
private NamedList deleteByQueryResponse = null;
private CharsRef scratch;
private final SchemaField idField;
private final SolrCmdDistributor cmdDistrib;
private boolean zkEnabled = false;
private String collection;
private ZkController zkController;
// these are setup at the start of each request processing
// method in this update processor
private boolean isLeader = true;
private boolean forwardToLeader = false;
private List<Node> nodes;
public DistributedUpdateProcessor(SolrQueryRequest req,
SolrQueryResponse rsp, UpdateRequestProcessor next) {
super(next);
this.rsp = rsp;
this.next = next;
this.idField = req.getSchema().getUniqueKeyField();
// version init
this.updateHandler = req.getCore().getUpdateHandler();
this.ulog = updateHandler.getUpdateLog();
this.vinfo = ulog == null ? null : ulog.getVersionInfo();
versionsStored = this.vinfo != null && this.vinfo.getVersionField() != null;
returnVersions = versionsStored;
// TODO: better way to get the response, or pass back info to it?
SolrRequestInfo reqInfo = returnVersions ? SolrRequestInfo.getRequestInfo() : null;
this.req = req;
CoreDescriptor coreDesc = req.getCore().getCoreDescriptor();
this.zkEnabled = coreDesc.getCoreContainer().isZooKeeperAware();
//this.rsp = reqInfo != null ? reqInfo.getRsp() : null;
zkController = req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
CloudDescriptor cloudDesc = coreDesc.getCloudDescriptor();
if (cloudDesc != null) {
collection = cloudDesc.getCollectionName();
}
cmdDistrib = new SolrCmdDistributor();
}
private List<Node> setupRequest(int hash) {
List<Node> nodes = null;
// if we are in zk mode...
if (zkEnabled) {
// the leader is...
// TODO: if there is no leader, wait and look again
// TODO: we are reading the leader from zk every time - we should cache
// this and watch for changes?? Just pull it from ZkController cluster state probably?
String shardId = getShard(hash, collection, zkController.getCloudState()); // get the right shard based on the hash...
try {
// TODO: if we find out we cannot talk to zk anymore, we should probably realize we are not
// a leader anymore - we shouldn't accept updates at all??
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(zkController.getZkStateReader().getLeaderProps(
collection, shardId));
String leaderNodeName = leaderProps.getCoreNodeName();
String coreName = req.getCore().getName();
String coreNodeName = zkController.getNodeName() + "_" + coreName;
isLeader = coreNodeName.equals(leaderNodeName);
if (req.getParams().getBool(SEEN_LEADER, false)) {
// we are coming from the leader, just go local - add no urls
forwardToLeader = false;
} else if (isLeader) {
// that means I want to forward onto my replicas...
// so get the replicas...
forwardToLeader = false;
List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
.getReplicaProps(collection, shardId, zkController.getNodeName(),
coreName);
if (replicaProps != null) {
nodes = new ArrayList<Node>(replicaProps.size());
for (ZkCoreNodeProps props : replicaProps) {
nodes.add(new StdNode(props));
}
}
} else {
// I need to forward onto the leader...
nodes = new ArrayList<Node>(1);
nodes.add(new RetryNode(leaderProps, zkController.getZkStateReader(), collection, shardId));
forwardToLeader = true;
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
}
}
return nodes;
}
private String getShard(int hash, String collection, CloudState cloudState) {
// ranges should be part of the cloud state and eventually gotten from zk
// get the shard names
return cloudState.getShard(hash, collection);
}
@Override
public void processAdd(AddUpdateCommand cmd) throws IOException {
// TODO: check for id field?
int hash = 0;
if (zkEnabled) {
zkCheck();
hash = hash(cmd);
nodes = setupRequest(hash);
} else {
// even in non zk mode, tests simulate updates from a leader
isLeader = !req.getParams().getBool(SEEN_LEADER, false);
}
boolean dropCmd = false;
if (!forwardToLeader) {
dropCmd = versionAdd(cmd);
}
if (dropCmd) {
// TODO: do we need to add anything to the response?
return;
}
ModifiableSolrParams params = null;
if (nodes != null) {
params = new ModifiableSolrParams(req.getParams());
if (isLeader) {
params.set(SEEN_LEADER, true);
}
cmdDistrib.distribAdd(cmd, nodes, params);
}
// TODO: what to do when no idField?
if (returnVersions && rsp != null && idField != null) {
if (addsResponse == null) {
addsResponse = new NamedList<String>();
rsp.add("adds",addsResponse);
}
if (scratch == null) scratch = new CharsRef();
idField.getType().indexedToReadable(cmd.getIndexedId(), scratch);
addsResponse.add(scratch.toString(), cmd.getVersion());
}
// TODO: keep track of errors? needs to be done at a higher level though since
// an id may fail before it gets to this processor.
// Given that, it may also make sense to move the version reporting out of this
// processor too.
}
// TODO: optionally fail if n replicas are not reached...
private void doFinish() {
// TODO: if not a forward and replication req is not specified, we could
// send in a background thread
cmdDistrib.finish();
Response response = cmdDistrib.getResponse();
// TODO - we may need to tell about more than one error...
// if its a forward, any fail is a problem -
// otherwise we assume things are fine if we got it locally
// until we start allowing min replication param
if (response.errors.size() > 0) {
// if one node is a RetryNode, this was a forward request
if (response.errors.get(0).node instanceof RetryNode) {
rsp.setException(response.errors.get(0).e);
}
// else
// for now we don't error - we assume if it was added locally, we
// succeeded
}
// if it is not a forward request, for each fail, try to tell them to
// recover - the doc was already added locally, so it should have been
// legit
// TODO: we should do this in the background it would seem
for (SolrCmdDistributor.Error error : response.errors) {
if (error.node instanceof RetryNode) {
// we don't try to force a leader to recover
// when we cannot forward to it
continue;
}
// TODO: we should force their state to recovering ??
// TODO: could be sent in parallel
// TODO: do retries??
// TODO: what if its is already recovering? Right now recoveries queue up -
// should they?
String recoveryUrl = error.node.getBaseUrl();
CommonsHttpSolrServer server;
log.info("try and ask " + recoveryUrl + " to recover");
try {
server = new CommonsHttpSolrServer(recoveryUrl);
server.setSoTimeout(5000);
server.setConnectionTimeout(5000);
RequestRecovery recoverRequestCmd = new RequestRecovery();
recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
recoverRequestCmd.setCoreName(error.node.getCoreName());
server.request(recoverRequestCmd);
} catch (Exception e) {
log.info("Could not tell a replica to recover", e);
}
}
}
// must be synchronized by bucket
private void doLocalAdd(AddUpdateCommand cmd) throws IOException {
super.processAdd(cmd);
}
// must be synchronized by bucket
private void doLocalDelete(DeleteUpdateCommand cmd) throws IOException {
super.processDelete(cmd);
}
/**
* @param cmd
* @return whether or not to drop this cmd
* @throws IOException
*/
private boolean versionAdd(AddUpdateCommand cmd) throws IOException {
BytesRef idBytes = cmd.getIndexedId();
if (vinfo == null || idBytes == null) {
super.processAdd(cmd);
return false;
}
// This is only the hash for the bucket, and must be based only on the uniqueKey (i.e. do not use a pluggable hash here)
int bucketHash = Hash.murmurhash3_x86_32(idBytes.bytes, idBytes.offset, idBytes.length, 0);
// at this point, there is an update we need to try and apply.
// we may or may not be the leader.
// Find any existing version in the document
// TODO: don't reuse update commands any more!
long versionOnUpdate = cmd.getVersion();
if (versionOnUpdate == 0) {
SolrInputField versionField = cmd.getSolrInputDocument().getField(VersionInfo.VERSION_FIELD);
if (versionField != null) {
Object o = versionField.getValue();
versionOnUpdate = o instanceof Number ? ((Number) o).longValue() : Long.parseLong(o.toString());
} else {
// Find the version
String versionOnUpdateS = req.getParams().get(VERSION_FIELD);
versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS);
}
}
boolean isReplay = (cmd.getFlags() & UpdateCommand.REPLAY) != 0;
boolean leaderLogic = isLeader && !isReplay;
VersionBucket bucket = vinfo.bucket(bucketHash);
vinfo.lockForUpdate();
try {
synchronized (bucket) {
// we obtain the version when synchronized and then do the add so we can ensure that
// if version1 < version2 then version1 is actually added before version2.
// even if we don't store the version field, synchronizing on the bucket
// will enable us to know what version happened first, and thus enable
// realtime-get to work reliably.
// TODO: if versions aren't stored, do we need to set on the cmd anyway for some reason?
// there may be other reasons in the future for a version on the commands
if (versionsStored) {
long bucketVersion = bucket.highest;
if (leaderLogic) {
long version = vinfo.getNewClock();
cmd.setVersion(version);
cmd.getSolrInputDocument().setField(VersionInfo.VERSION_FIELD, version);
bucket.updateHighest(version);
} else {
// The leader forwarded us this update.
cmd.setVersion(versionOnUpdate);
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
// we're not in an active state, and this update isn't from a replay, so buffer it.
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
ulog.add(cmd);
return true;
}
// if we aren't the leader, then we need to check that updates were not re-ordered
if (bucketVersion != 0 && bucketVersion < versionOnUpdate) {
// we're OK... this update has a version higher than anything we've seen
// in this bucket so far, so we know that no reordering has yet occured.
bucket.updateHighest(versionOnUpdate);
} else {
// there have been updates higher than the current update. we need to check
// the specific version for this id.
Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId());
if (lastVersion != null && Math.abs(lastVersion) >= versionOnUpdate) {
// This update is a repeat, or was reordered. We need to drop this update.
return true;
}
}
}
}
doLocalAdd(cmd);
} // end synchronized (bucket)
} finally {
vinfo.unlockForUpdate();
}
return false;
}
@Override
public void processDelete(DeleteUpdateCommand cmd) throws IOException {
if (!cmd.isDeleteById()) {
// delete by query...
// TODO: handle versioned and distributed deleteByQuery
// even in non zk mode, tests simulate updates from a leader
if(!zkEnabled) {
isLeader = !req.getParams().getBool(SEEN_LEADER, false);
} else {
zkCheck();
}
processDeleteByQuery(cmd);
return;
}
int hash = 0;
if (zkEnabled) {
zkCheck();
hash = hash(cmd);
nodes = setupRequest(hash);
} else {
// even in non zk mode, tests simulate updates from a leader
isLeader = !req.getParams().getBool(SEEN_LEADER, false);
}
boolean dropCmd = false;
if (!forwardToLeader) {
dropCmd = versionDelete(cmd);
}
if (dropCmd) {
// TODO: do we need to add anything to the response?
return;
}
ModifiableSolrParams params = null;
if (nodes != null) {
params = new ModifiableSolrParams(req.getParams());
if (isLeader) {
params.set(SEEN_LEADER, true);
}
cmdDistrib.distribDelete(cmd, nodes, params);
}
// cmd.getIndexId == null when delete by query
// TODO: what to do when no idField?
if (returnVersions && rsp != null && cmd.getIndexedId() != null && idField != null) {
if (deleteResponse == null) {
deleteResponse = new NamedList<String>();
rsp.add("deletes",deleteResponse);
}
if (scratch == null) scratch = new CharsRef();
idField.getType().indexedToReadable(cmd.getIndexedId(), scratch);
deleteResponse.add(scratch.toString(), cmd.getVersion()); // we're returning the version of the delete.. not the version of the doc we deleted.
}
}
private void zkCheck() {
int retries = 10;
while (!zkController.isConnected()) {
if (retries-- == 0) {
throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Cannot talk to ZooKeeper - Updates are disabled.");
}
try {
Thread.sleep(100);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
break;
}
}
}
private boolean versionDelete(DeleteUpdateCommand cmd) throws IOException {
BytesRef idBytes = cmd.getIndexedId();
if (vinfo == null || idBytes == null) {
super.processDelete(cmd);
return false;
}
// This is only the hash for the bucket, and must be based only on the uniqueKey (i.e. do not use a pluggable hash here)
int bucketHash = Hash.murmurhash3_x86_32(idBytes.bytes, idBytes.offset, idBytes.length, 0);
// at this point, there is an update we need to try and apply.
// we may or may not be the leader.
// Find the version
long versionOnUpdate = cmd.getVersion();
if (versionOnUpdate == 0) {
String versionOnUpdateS = req.getParams().get(VERSION_FIELD);
versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS);
}
versionOnUpdate = Math.abs(versionOnUpdate); // normalize to positive version
boolean isReplay = (cmd.getFlags() & UpdateCommand.REPLAY) != 0;
boolean leaderLogic = isLeader && !isReplay;
if (!leaderLogic && versionOnUpdate==0) {
throw new SolrException(ErrorCode.BAD_REQUEST, "missing _version_ on update from leader");
}
VersionBucket bucket = vinfo.bucket(bucketHash);
vinfo.lockForUpdate();
try {
synchronized (bucket) {
if (versionsStored) {
long bucketVersion = bucket.highest;
if (leaderLogic) {
long version = vinfo.getNewClock();
cmd.setVersion(-version);
bucket.updateHighest(version);
} else {
cmd.setVersion(-versionOnUpdate);
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
// we're not in an active state, and this update isn't from a replay, so buffer it.
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
ulog.delete(cmd);
return true;
}
// if we aren't the leader, then we need to check that updates were not re-ordered
if (bucketVersion != 0 && bucketVersion < versionOnUpdate) {
// we're OK... this update has a version higher than anything we've seen
// in this bucket so far, so we know that no reordering has yet occured.
bucket.updateHighest(versionOnUpdate);
} else {
// there have been updates higher than the current update. we need to check
// the specific version for this id.
Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId());
if (lastVersion != null && Math.abs(lastVersion) >= versionOnUpdate) {
// This update is a repeat, or was reordered. We need to drop this update.
return true;
}
}
}
}
doLocalDelete(cmd);
return false;
} // end synchronized (bucket)
} finally {
vinfo.unlockForUpdate();
}
}
private void processDeleteByQuery(DeleteUpdateCommand cmd) throws IOException {
if (vinfo == null) {
super.processDelete(cmd);
return;
}
// at this point, there is an update we need to try and apply.
// we may or may not be the leader.
// Find the version
long versionOnUpdate = cmd.getVersion();
if (versionOnUpdate == 0) {
String versionOnUpdateS = req.getParams().get(VERSION_FIELD);
versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS);
}
versionOnUpdate = Math.abs(versionOnUpdate); // normalize to positive version
boolean isReplay = (cmd.getFlags() & UpdateCommand.REPLAY) != 0;
boolean leaderLogic = isLeader && !isReplay;
if (!leaderLogic && versionOnUpdate==0) {
throw new SolrException(ErrorCode.BAD_REQUEST, "missing _version_ on update from leader");
}
vinfo.blockUpdates();
try {
if (versionsStored) {
if (leaderLogic) {
long version = vinfo.getNewClock();
cmd.setVersion(-version);
// TODO update versions in all buckets
} else {
cmd.setVersion(-versionOnUpdate);
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
// we're not in an active state, and this update isn't from a replay, so buffer it.
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
ulog.deleteByQuery(cmd);
return;
}
}
}
doLocalDelete(cmd);
// since we don't know which documents were deleted, the easiest thing to do is to invalidate
// all real-time caches (i.e. UpdateLog) which involves also getting a new version of the IndexReader
// (so cache misses will see up-to-date data)
} finally {
vinfo.unblockUpdates();
}
// TODO: we should consider this? Send delete query to everyone in the current collection
if (zkEnabled) {
ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
if (!params.getBool(DELQUERY_END_POINT, false)) {
params.set(DELQUERY_END_POINT, true);
String nodeName = req.getCore().getCoreDescriptor().getCoreContainer()
.getZkController().getNodeName();
String shardZkNodeName = nodeName + "_" + req.getCore().getName();
List<Node> nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
.getCloudDescriptor().getCollectionName(), shardZkNodeName);
if (nodes != null) {
cmdDistrib.distribDelete(cmd, nodes, params);
finish();
}
}
}
if (returnVersions && rsp != null) {
if (deleteByQueryResponse == null) {
deleteByQueryResponse = new NamedList<String>();
rsp.add("deleteByQuery",deleteByQueryResponse);
}
deleteByQueryResponse.add(cmd.getQuery(), cmd.getVersion());
}
}
@Override
public void processCommit(CommitUpdateCommand cmd) throws IOException {
if (zkEnabled) {
zkCheck();
}
if (vinfo != null) {
vinfo.lockForUpdate();
}
try {
if (ulog == null || ulog.getState() == UpdateLog.State.ACTIVE || (cmd.getFlags() & UpdateCommand.REPLAY) != 0) {
super.processCommit(cmd);
} else {
log.info("Ignoring commit while not ACTIVE - state: " + ulog.getState() + " replay:" + (cmd.getFlags() & UpdateCommand.REPLAY));
}
} finally {
if (vinfo != null) {
vinfo.unlockForUpdate();
}
}
// TODO: we should consider this? commit everyone in the current collection
if (zkEnabled) {
ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
if (!params.getBool(COMMIT_END_POINT, false)) {
params.set(COMMIT_END_POINT, true);
String nodeName = req.getCore().getCoreDescriptor().getCoreContainer()
.getZkController().getNodeName();
String shardZkNodeName = nodeName + "_" + req.getCore().getName();
List<Node> nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
.getCloudDescriptor().getCollectionName(), shardZkNodeName);
if (nodes != null) {
cmdDistrib.distribCommit(cmd, nodes, params);
finish();
}
}
}
}
@Override
public void finish() throws IOException {
doFinish();
if (next != null && nodes == null) next.finish();
}
private List<Node> getCollectionUrls(SolrQueryRequest req, String collection, String shardZkNodeName) {
CloudState cloudState = req.getCore().getCoreDescriptor()
.getCoreContainer().getZkController().getCloudState();
List<Node> urls = new ArrayList<Node>();
Map<String,Slice> slices = cloudState.getSlices(collection);
if (slices == null) {
throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
"Could not find collection in zk: " + cloudState);
}
for (Map.Entry<String,Slice> sliceEntry : slices.entrySet()) {
Slice replicas = slices.get(sliceEntry.getKey());
Map<String,ZkNodeProps> shardMap = replicas.getShards();
for (Entry<String,ZkNodeProps> entry : shardMap.entrySet()) {
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
if (cloudState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(shardZkNodeName)) {
urls.add(new StdNode(nodeProps));
}
}
}
if (urls.size() == 0) {
return null;
}
return urls;
}
// TODO: move this to AddUpdateCommand/DeleteUpdateCommand and cache it? And
// make the hash pluggable of course.
// The hash also needs to be pluggable
private int hash(AddUpdateCommand cmd) {
BytesRef br = cmd.getIndexedId();
return Hash.murmurhash3_x86_32(br.bytes, br.offset, br.length, 0);
}
private int hash(DeleteUpdateCommand cmd) {
BytesRef br = cmd.getIndexedId();
return Hash.murmurhash3_x86_32(br.bytes, br.offset, br.length, 0);
}
// RetryNodes are used in the case of 'forward to leader' where we want
// to try the latest leader on a fail in the case the leader just went down.
public static class RetryNode extends StdNode {
private ZkStateReader zkStateReader;
private String collection;
private String shardId;
public RetryNode(ZkCoreNodeProps nodeProps, ZkStateReader zkStateReader, String collection, String shardId) {
super(nodeProps);
this.zkStateReader = zkStateReader;
this.collection = collection;
this.shardId = shardId;
}
@Override
public String toString() {
return url;
}
@Override
public boolean checkRetry() {
ZkCoreNodeProps leaderProps;
try {
leaderProps = new ZkCoreNodeProps(zkStateReader.getLeaderProps(
collection, shardId));
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return false;
}
this.url = leaderProps.getCoreUrl();
return true;
}
@Override
public int hashCode() {
final int prime = 31;
int result = super.hashCode();
result = prime * result
+ ((collection == null) ? 0 : collection.hashCode());
result = prime * result + ((shardId == null) ? 0 : shardId.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!super.equals(obj)) return false;
if (getClass() != obj.getClass()) return false;
RetryNode other = (RetryNode) obj;
if (url == null) {
if (other.url != null) return false;
} else if (!url.equals(other.url)) return false;
return true;
}
}
}

View File

@ -0,0 +1,40 @@
package org.apache.solr.update.processor;
/**
* 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.util.NamedList;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
public class DistributedUpdateProcessorFactory extends
UpdateRequestProcessorFactory {
@Override
public void init(NamedList args) {
}
@Override
public DistributedUpdateProcessor getInstance(SolrQueryRequest req,
SolrQueryResponse rsp, UpdateRequestProcessor next) {
return new DistributedUpdateProcessor(req, rsp, next);
}
}

View File

@ -117,20 +117,20 @@ class LogUpdateProcessor extends UpdateRequestProcessor {
@Override
public void processDelete( DeleteUpdateCommand cmd ) throws IOException {
if (cmd.id != null) {
if (cmd.isDeleteById()) {
if (deletes == null) {
deletes = new ArrayList<String>();
toLog.add("delete",deletes);
}
if (deletes.size() < maxNumToLog) {
deletes.add(cmd.id);
deletes.add(cmd.getId());
}
if (logDebug) { log.debug("delete {}", cmd.id); }
if (logDebug) { log.debug("delete {}", cmd.getId()); }
} else {
if (toLog.size() < maxNumToLog) {
toLog.add("deleteByQuery", cmd.query);
}
if (logDebug) { log.debug("deleteByQuery {}", cmd.query); }
if (logDebug) { log.debug("deleteByQuery {}", cmd.getQuery()); }
}
numDeletes++;

View File

@ -21,13 +21,7 @@ import java.io.IOException;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.update.AddUpdateCommand;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.DeleteUpdateCommand;
import org.apache.solr.update.DocumentBuilder;
import org.apache.solr.update.MergeIndexesCommand;
import org.apache.solr.update.RollbackUpdateCommand;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.*;
/**
@ -49,6 +43,8 @@ class RunUpdateProcessor extends UpdateRequestProcessor
private final SolrQueryRequest req;
private final UpdateHandler updateHandler;
private boolean changesSinceCommit = false;
public RunUpdateProcessor(SolrQueryRequest req, UpdateRequestProcessor next) {
super( next );
this.req = req;
@ -59,17 +55,19 @@ class RunUpdateProcessor extends UpdateRequestProcessor
public void processAdd(AddUpdateCommand cmd) throws IOException {
updateHandler.addDoc(cmd);
super.processAdd(cmd);
changesSinceCommit = true;
}
@Override
public void processDelete(DeleteUpdateCommand cmd) throws IOException {
if( cmd.id != null ) {
if( cmd.isDeleteById()) {
updateHandler.delete(cmd);
}
else {
updateHandler.deleteByQuery(cmd);
}
super.processDelete(cmd);
changesSinceCommit = true;
}
@Override
@ -83,6 +81,7 @@ class RunUpdateProcessor extends UpdateRequestProcessor
{
updateHandler.commit(cmd);
super.processCommit(cmd);
changesSinceCommit = false;
}
/**
@ -93,6 +92,16 @@ class RunUpdateProcessor extends UpdateRequestProcessor
{
updateHandler.rollback(cmd);
super.processRollback(cmd);
changesSinceCommit = false;
}
@Override
public void finish() throws IOException {
if (changesSinceCommit && updateHandler.getUpdateLog() != null) {
updateHandler.getUpdateLog().finish(null);
}
super.finish();
}
}

View File

@ -0,0 +1,49 @@
package org.apache.solr.util;
/**
* 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.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
public class DefaultSolrThreadFactory implements ThreadFactory {
private static final AtomicInteger poolNumber = new AtomicInteger(1);
private final ThreadGroup group;
private final AtomicInteger threadNumber = new AtomicInteger(1);
private final String prefix;
public DefaultSolrThreadFactory(String namePrefix) {
SecurityManager s = System.getSecurityManager();
group = (s != null)? s.getThreadGroup() :
Thread.currentThread().getThreadGroup();
prefix = namePrefix + "-" +
poolNumber.getAndIncrement() +
"-thread-";
}
public Thread newThread(Runnable r) {
Thread t = new Thread(group, r,
prefix + threadNumber.getAndIncrement(),
0);
t.setDaemon(false);
if (t.getPriority() != Thread.NORM_PRIORITY)
t.setPriority(Thread.NORM_PRIORITY);
return t;
}
}

View File

@ -560,6 +560,8 @@
<field name="tlong" type="tlong" indexed="true" stored="true" />
<field name="_version_" type="long" indexed="true" stored="true" multiValued="false"/>
<!-- Dynamic field definitions. If a field name is not found, dynamicFields
will be used if the name matches any of the patterns.
RESTRICTION: the glob-like pattern in the name attribute must have

View File

@ -530,6 +530,11 @@
<field name="uniq3" type="string" indexed="true" stored="true"/>
<field name="nouniq" type="string" indexed="true" stored="true" multiValued="true"/>
<!-- for versioning -->
<field name="_version_" type="long" indexed="true" stored="true"/>
<dynamicField name="*_coordinate" type="tdouble" indexed="true" stored="false"/>

View File

@ -21,6 +21,7 @@
DO NOT ADD THINGS TO THIS CONFIG! -->
<config>
<luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
<dataDir>${solr.data.dir:}</dataDir>
<directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
<requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
</config>

View File

@ -30,8 +30,11 @@
solr.RAMDirectoryFactory is memory based and not persistent. -->
<directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
<dataDir>${solr.data.dir:}</dataDir>
<updateHandler class="solr.DirectUpdateHandler2">
</updateHandler>
<requestHandler name="standard" class="solr.StandardRequestHandler"/>
<requestDispatcher handleSelect="true" >

View File

@ -19,7 +19,9 @@
<config>
<luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
<directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
<dataDir>${solr.data.dir:}</dataDir>
<requestHandler name="standard" class="solr.StandardRequestHandler">
</requestHandler>
@ -29,8 +31,14 @@
</lst>
</requestHandler>
<requestHandler name="/update" class="solr.XmlUpdateRequestHandler">
</requestHandler>
<requestHandler name="/update/json" class="solr.JsonUpdateRequestHandler">
</requestHandler>
<updateHandler class="solr.DirectUpdateHandler2">
<updateLog class="solr.FSUpdateLog">
<updateLog>
<!-- <str name="dir">/tmp/solr/</str> -->
</updateLog>
</updateHandler>

View File

@ -142,6 +142,9 @@
</listener>
-->
<updateLog enable="${enable.update.log:false}">
<str name="dir">${solr.data.dir:}</str>
</updateLog>
</updateHandler>
@ -235,6 +238,7 @@
</query>
<requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
<!-- An alternate set representation that uses an integer hash to store filters (sets of docids).
If the set cardinality <= maxSize elements, then HashDocSet will be used instead of the bitset

View File

@ -28,7 +28,7 @@
adminPath: RequestHandler path to manage cores.
If 'null' (or absent), cores will not be manageable via request handler
-->
<cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" hostContext="solr" zkClientTimeout="8000">
<core name="collection1" shard="${shard:}" collection="${collection:collection1}" config="${solrconfig:solrconfig.xml}" instanceDir="."/>
<cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" hostContext="solr" zkClientTimeout="12000" numShards="${numShards:3}">
<core name="collection1" shard="${shard:}" collection="${collection:collection1}" config="${solrconfig:solrconfig.xml}" schema="${schema:schema.xml}" instanceDir="."/>
</cores>
</solr>

View File

@ -121,7 +121,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
// test merge factor picked up
SolrCore core = h.getCore();
IndexWriter writer = ((DirectUpdateHandler2)core.getUpdateHandler()).getIndexWriterProvider().getIndexWriter(core);
IndexWriter writer = ((DirectUpdateHandler2)core.getUpdateHandler()).getSolrCoreState().getIndexWriter(core);
assertEquals("Mergefactor was not picked up", ((LogMergePolicy)writer.getConfig().getMergePolicy()).getMergeFactor(), 8);
lrf.args.put(CommonParams.VERSION,"2.2");

View File

@ -45,10 +45,9 @@ public class TestSolrCoreProperties extends LuceneTestCase {
public void setUp() throws Exception {
super.setUp();
setUpMe();
System.setProperty("solr.solr.home", getHomeDir());
System.setProperty("solr.data.dir", getDataDir());
solrJetty = new JettySolrRunner("/solr", 0);
solrJetty = new JettySolrRunner(getHomeDir(), "/solr", 0);
solrJetty.start();
String url = "http://localhost:" + solrJetty.getLocalPort() + "/solr";

View File

@ -23,6 +23,7 @@ import org.apache.solr.BaseDistributedSearchTestCase;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.core.SolrConfig;
import org.junit.AfterClass;
import org.junit.Before;
public abstract class AbstractDistributedZkTestCase extends BaseDistributedSearchTestCase {
@ -34,7 +35,7 @@ public abstract class AbstractDistributedZkTestCase extends BaseDistributedSearc
public void setUp() throws Exception {
super.setUp();
log.info("####SETUP_START " + getName());
createTempDir();
ignoreException("java.nio.channels.ClosedChannelException");
String zkDir = testDir.getAbsolutePath() + File.separator
@ -80,8 +81,8 @@ public abstract class AbstractDistributedZkTestCase extends BaseDistributedSearc
System.clearProperty("collection");
System.clearProperty("solr.test.sys.prop1");
System.clearProperty("solr.test.sys.prop2");
super.tearDown();
resetExceptionIgnores();
super.tearDown();
}
protected void printLayout() throws Exception {
@ -89,4 +90,10 @@ public abstract class AbstractDistributedZkTestCase extends BaseDistributedSearc
zkClient.printLayoutToStdOut();
zkClient.close();
}
@AfterClass
public static void afterClass() throws InterruptedException {
// wait just a bit for any zk client threads to outlast timeout
Thread.sleep(2000);
}
}

View File

@ -18,12 +18,15 @@ package org.apache.solr.cloud;
*/
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrConfig;
import org.apache.zookeeper.CreateMode;
import org.junit.AfterClass;
@ -56,6 +59,7 @@ public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
zkServer = new ZkTestServer(zkDir);
zkServer.run();
System.setProperty("solrcloud.skip.autorecovery", "true");
System.setProperty("zkHost", zkServer.getZkAddress());
System.setProperty("hostPort", "0000");
@ -69,21 +73,23 @@ public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
static void buildZooKeeper(String zkHost, String zkAddress, String config,
String schema) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, AbstractZkTestCase.TIMEOUT);
zkClient.makePath("/solr");
zkClient.makePath("/solr", false, true);
zkClient.close();
zkClient = new SolrZkClient(zkAddress, AbstractZkTestCase.TIMEOUT);
ZkNodeProps props = new ZkNodeProps();
Map<String,String> props = new HashMap<String,String>();
props.put("configName", "conf1");
zkClient.makePath("/collections/collection1", props.store(), CreateMode.PERSISTENT);
zkClient.makePath("/collections/collection1/shards", CreateMode.PERSISTENT);
final ZkNodeProps zkProps = new ZkNodeProps(props);
zkClient.makePath("/collections/control_collection", props.store(), CreateMode.PERSISTENT);
zkClient.makePath("/collections/control_collection/shards", CreateMode.PERSISTENT);
zkClient.makePath("/collections/collection1", ZkStateReader.toJSON(zkProps), CreateMode.PERSISTENT, true);
zkClient.makePath("/collections/collection1/shards", CreateMode.PERSISTENT, true);
zkClient.makePath("/collections/control_collection", ZkStateReader.toJSON(zkProps), CreateMode.PERSISTENT, true);
zkClient.makePath("/collections/control_collection/shards", CreateMode.PERSISTENT, true);
putConfig(zkClient, config);
putConfig(zkClient, schema);
putConfig(zkClient, "solrconfig.xml");
putConfig(zkClient, "stopwords.txt");
putConfig(zkClient, "protwords.txt");
putConfig(zkClient, "mapping-ISOLatin1Accent.txt");
@ -93,10 +99,10 @@ public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
zkClient.close();
}
private static void putConfig(SolrZkClient zkConnection, String name)
private static void putConfig(SolrZkClient zkClient, final String name)
throws Exception {
zkConnection.setData("/configs/conf1/" + name, getFile("solr"
+ File.separator + "conf" + File.separator + name));
zkClient.makePath("/configs/conf1/" + name, getFile("solr"
+ File.separator + "conf" + File.separator + name), false, false);
}
@Override
@ -109,11 +115,15 @@ public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
}
@AfterClass
public static void azt_afterClass() throws IOException {
zkServer.shutdown();
public static void azt_afterClass() throws Exception {
System.clearProperty("zkHost");
System.clearProperty("solr.test.sys.prop1");
System.clearProperty("solr.test.sys.prop2");
System.clearProperty("solrcloud.skip.autorecovery");
zkServer.shutdown();
// wait just a bit for any zk client threads to outlast timeout
Thread.sleep(2000);
}
protected void printLayout(String zkHost) throws Exception {
@ -122,24 +132,24 @@ public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
zkClient.close();
}
static void makeSolrZkNode(String zkHost) throws Exception {
public static void makeSolrZkNode(String zkHost) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, TIMEOUT);
zkClient.makePath("/solr");
zkClient.makePath("/solr", false, true);
zkClient.close();
}
static void tryCleanSolrZkNode(String zkHost) throws Exception {
public static void tryCleanSolrZkNode(String zkHost) throws Exception {
tryCleanPath(zkHost, "/solr");
}
static void tryCleanPath(String zkHost, String path) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, TIMEOUT);
if (zkClient.exists(path)) {
List<String> children = zkClient.getChildren(path, null);
if (zkClient.exists(path, true)) {
List<String> children = zkClient.getChildren(path, null, true);
for (String string : children) {
tryCleanPath(zkHost, path+"/"+string);
}
zkClient.delete(path, -1);
zkClient.delete(path, -1, true);
}
zkClient.close();
}

View File

@ -17,15 +17,24 @@ package org.apache.solr.cloud;
* limitations under the License.
*/
import java.io.File;
import java.io.IOException;
import java.net.MalformedURLException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrServer;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.junit.BeforeClass;
/**
*
@ -51,16 +60,10 @@ public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
String missingField="ignore_exception__missing_but_valid_field_t";
String invalidField="ignore_exception__invalid_field_not_in_schema";
private Map<String,List<SolrServer>> otherCollectionClients = new HashMap<String,List<SolrServer>>();
public BasicDistributedZkTest() {
fixShardCount = true;
System.setProperty("CLOUD_UPDATE_DELAY", "0");
}
@BeforeClass
public static void beforeClass() throws Exception {
System.setProperty("solr.solr.home", SolrTestCaseJ4.TEST_HOME());
}
@Override
@ -68,7 +71,6 @@ public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
if (r.nextBoolean()) {
// don't set shards, let that be figured out from the cloud state
params.set("distrib", "true");
} else {
// use shard ids rather than physical locations
StringBuilder sb = new StringBuilder();
@ -78,7 +80,6 @@ public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
sb.append("shard" + (i + 3));
}
params.set("shards", sb.toString());
params.set("distrib", "true");
}
}
@ -241,12 +242,105 @@ public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
// on shards with matches.
// query("q","matchesnothing","fl","*,score", "debugQuery", "true");
testMultipleCollections();
// Thread.sleep(10000000000L);
if (DEBUG) {
super.printLayout();
}
}
private void testMultipleCollections() throws MalformedURLException,
SolrServerException, IOException, Exception {
// create another 2 collections and search across them
createNewCollection("collection2");
indexDoc("collection2", getDoc(id, "10000000"));
indexDoc("collection2", getDoc(id, "10000001"));
indexDoc("collection2", getDoc(id, "10000003"));
createNewCollection("collection3");
indexDoc("collection3", getDoc(id, "20000000"));
indexDoc("collection3", getDoc(id, "20000001"));
otherCollectionClients.get("collection2").get(0).commit();
otherCollectionClients.get("collection3").get(0).commit();
long collection1Docs = solrj.query(new SolrQuery("*:*")).getResults()
.getNumFound();
long collection2Docs = otherCollectionClients.get("collection2").get(0)
.query(new SolrQuery("*:*")).getResults().getNumFound();
long collection3Docs = otherCollectionClients.get("collection3").get(0)
.query(new SolrQuery("*:*")).getResults().getNumFound();
SolrQuery query = new SolrQuery("*:*");
query.set("collection", "collection2,collection3");
long found = clients.get(0).query(query).getResults().getNumFound();
assertEquals(collection2Docs + collection3Docs, found);
query = new SolrQuery("*:*");
query.set("collection", "collection1,collection2,collection3");
found = clients.get(0).query(query).getResults().getNumFound();
assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
// try to search multiple with cloud client
found = solrj.query(query).getResults().getNumFound();
assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
query.set("collection", "collection2,collection3");
found = solrj.query(query).getResults().getNumFound();
assertEquals(collection2Docs + collection3Docs, found);
query.set("collection", "collection3");
found = solrj.query(query).getResults().getNumFound();
assertEquals(collection3Docs, found);
query.remove("collection");
found = solrj.query(query).getResults().getNumFound();
assertEquals(collection1Docs, found);
}
protected SolrInputDocument getDoc(Object... fields) throws Exception {
SolrInputDocument doc = new SolrInputDocument();
addFields(doc, fields);
return doc;
}
protected void indexDoc(String collection, SolrInputDocument doc) throws IOException, SolrServerException {
List<SolrServer> clients = otherCollectionClients.get(collection);
int which = (doc.getField(id).toString().hashCode() & 0x7fffffff) % clients.size();
SolrServer client = clients.get(which);
client.add(doc);
}
private void createNewCollection(String collection)
throws MalformedURLException, SolrServerException, IOException {
List<SolrServer> collectionClients = new ArrayList<SolrServer>();
otherCollectionClients.put(collection, collectionClients);
for (SolrServer client : clients) {
CommonsHttpSolrServer server = new CommonsHttpSolrServer(
((CommonsHttpSolrServer) client).getBaseURL());
Create createCmd = new Create();
createCmd.setCoreName(collection);
createCmd.setDataDir(dataDir.getAbsolutePath() + File.separator + collection);
server.request(createCmd);
collectionClients.add(createNewSolrServer(collection,
((CommonsHttpSolrServer) client).getBaseURL()));
}
}
protected SolrServer createNewSolrServer(String collection, String baseUrl) {
try {
// setup the server...
CommonsHttpSolrServer s = new CommonsHttpSolrServer(baseUrl + "/" + collection);
s.setConnectionTimeout(100); // 1/10th sec
s.setDefaultMaxConnectionsPerHost(100);
s.setMaxTotalConnections(100);
return s;
}
catch (Exception ex) {
throw new RuntimeException(ex);
}
}
volatile CloudSolrServer solrj;
@Override
@ -278,7 +372,9 @@ public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
@Override
public void tearDown() throws Exception {
super.tearDown();
System.clearProperty("CLOUD_UPDATE_DELAY");
if (solrj != null) {
solrj.close();
}
System.clearProperty("zkHost");
}
}

View File

@ -21,7 +21,10 @@ import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.update.DirectUpdateHandler2;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -29,13 +32,16 @@ import org.junit.Test;
import org.xml.sax.SAXParseException;
/**
*
* This test is not fully functional - the port registered is illegal -
* so you cannot hit this with http - a nice side benifit is that it will
* detect if a node is trying to do an update to itself with http - it shouldn't
* do that.
*/
public class BasicZkTest extends AbstractZkTestCase {
@BeforeClass
public static void beforeClass() {
System.setProperty("CLOUD_UPDATE_DELAY", "1");
}
@Test
@ -48,12 +54,12 @@ public class BasicZkTest extends AbstractZkTestCase {
// test merge factor picked up
SolrCore core = h.getCore();
IndexWriter writer = ((DirectUpdateHandler2)core.getUpdateHandler()).getIndexWriterProvider().getIndexWriter(core);
IndexWriter writer = ((DirectUpdateHandler2)core.getUpdateHandler()).getSolrCoreState().getIndexWriter(core);
assertEquals("Mergefactor was not picked up", ((LogMergePolicy)writer.getConfig().getMergePolicy()).getMergeFactor(), 8);
lrf.args.put(CommonParams.VERSION, "2.2");
assertQ("test query on empty index", req("qlkciyopsbgzyvkylsjhchghjrdf"),
assertQ("test query on empty index", request("qlkciyopsbgzyvkylsjhchghjrdf"),
"//result[@numFound='0']");
// test escaping of ";"
@ -61,29 +67,29 @@ public class BasicZkTest extends AbstractZkTestCase {
assertU("adding doc#42", adoc("id", "42", "val_s", "aa;bb"));
assertU("does commit work?", commit());
assertQ("backslash escaping semicolon", req("id:42 AND val_s:aa\\;bb"),
assertQ("backslash escaping semicolon", request("id:42 AND val_s:aa\\;bb"),
"//*[@numFound='1']", "//int[@name='id'][.='42']");
assertQ("quote escaping semicolon", req("id:42 AND val_s:\"aa;bb\""),
assertQ("quote escaping semicolon", request("id:42 AND val_s:\"aa;bb\""),
"//*[@numFound='1']", "//int[@name='id'][.='42']");
assertQ("no escaping semicolon", req("id:42 AND val_s:aa"),
assertQ("no escaping semicolon", request("id:42 AND val_s:aa"),
"//*[@numFound='0']");
assertU(delI("42"));
assertU(commit());
assertQ(req("id:42"), "//*[@numFound='0']");
assertQ(request("id:42"), "//*[@numFound='0']");
// test overwrite default of true
assertU(adoc("id", "42", "val_s", "AAA"));
assertU(adoc("id", "42", "val_s", "BBB"));
assertU(commit());
assertQ(req("id:42"), "//*[@numFound='1']", "//str[.='BBB']");
assertQ(request("id:42"), "//*[@numFound='1']", "//str[.='BBB']");
assertU(adoc("id", "42", "val_s", "CCC"));
assertU(adoc("id", "42", "val_s", "DDD"));
assertU(commit());
assertQ(req("id:42"), "//*[@numFound='1']", "//str[.='DDD']");
assertQ(request("id:42"), "//*[@numFound='1']", "//str[.='DDD']");
// test deletes
String[] adds = new String[] { add(doc("id", "101"), "overwrite", "true"),
@ -115,28 +121,28 @@ public class BasicZkTest extends AbstractZkTestCase {
zkController.getNodeName()));
// test maxint
assertQ(req("q", "id:[100 TO 110]", "rows", "2147483647"),
assertQ(request("q", "id:[100 TO 110]", "rows", "2147483647"),
"//*[@numFound='4']");
// test big limit
assertQ(req("q", "id:[100 TO 111]", "rows", "1147483647"),
assertQ(request("q", "id:[100 TO 111]", "rows", "1147483647"),
"//*[@numFound='4']");
assertQ(req("id:[100 TO 110]"), "//*[@numFound='4']");
assertQ(request("id:[100 TO 110]"), "//*[@numFound='4']");
assertU(delI("102"));
assertU(commit());
assertQ(req("id:[100 TO 110]"), "//*[@numFound='3']");
assertQ(request("id:[100 TO 110]"), "//*[@numFound='3']");
assertU(delI("105"));
assertU(commit());
assertQ(req("id:[100 TO 110]"), "//*[@numFound='2']");
assertQ(request("id:[100 TO 110]"), "//*[@numFound='2']");
assertU(delQ("id:[100 TO 110]"));
assertU(commit());
assertQ(req("id:[100 TO 110]"), "//*[@numFound='0']");
assertQ(request("id:[100 TO 110]"), "//*[@numFound='0']");
// SOLR-2651: test that reload still gets config files from zookeeper
zkController.getZkClient().setData("/configs/conf1/solrconfig.xml", new byte[0]);
zkController.getZkClient().setData("/configs/conf1/solrconfig.xml", new byte[0], true);
// we set the solrconfig to nothing, so this reload should fail
try {
@ -150,8 +156,17 @@ public class BasicZkTest extends AbstractZkTestCase {
}
public SolrQueryRequest request(String... q) {
LocalSolrQueryRequest req = lrf.makeRequest(q);
ModifiableSolrParams params = new ModifiableSolrParams();
params.add(req.getParams());
params.set("distrib", false);
req.setParams(params);
return req;
}
@AfterClass
public static void afterClass() {
System.clearProperty("CLOUD_UPDATE_DELAY");
}
}

View File

@ -0,0 +1,434 @@
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.net.BindException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.FullSolrCloudTest.CloudJettyRunner;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.servlet.SolrDispatchFilter;
import org.apache.zookeeper.KeeperException;
import org.mortbay.jetty.servlet.FilterHolder;
/**
* The monkey can stop random or specific jetties used with SolrCloud.
*
* It can also run in a background thread and start and stop jetties
* randomly.
*
*/
public class ChaosMonkey {
private static final int CONLOSS_PERCENT = 3; //30%
private static final int EXPIRE_PERCENT = 4; //40%
private Map<String,List<CloudJettyRunner>> shardToJetty;
private ZkTestServer zkServer;
private ZkStateReader zkStateReader;
private String collection;
private Random random;
private volatile boolean stop = false;
private AtomicInteger stops = new AtomicInteger();
private AtomicInteger starts = new AtomicInteger();
private AtomicInteger expires = new AtomicInteger();
private AtomicInteger connloss = new AtomicInteger();
private Map<String,List<SolrServer>> shardToClient;
private boolean expireSessions;
private boolean causeConnectionLoss;
private boolean aggressivelyKillLeaders;
private Map<String,SolrServer> shardToLeaderClient;
private Map<String,CloudJettyRunner> shardToLeaderJetty;
private long startTime;
public ChaosMonkey(ZkTestServer zkServer, ZkStateReader zkStateReader,
String collection, Map<String,List<CloudJettyRunner>> shardToJetty,
Map<String,List<SolrServer>> shardToClient,
Map<String,SolrServer> shardToLeaderClient,
Map<String,CloudJettyRunner> shardToLeaderJetty, Random random) {
this.shardToJetty = shardToJetty;
this.shardToClient = shardToClient;
this.shardToLeaderClient = shardToLeaderClient;
this.shardToLeaderJetty = shardToLeaderJetty;
this.zkServer = zkServer;
this.zkStateReader = zkStateReader;
this.collection = collection;
this.random = random;
expireSessions = random.nextBoolean();
causeConnectionLoss = random.nextBoolean();
}
public void expireSession(JettySolrRunner jetty) {
SolrDispatchFilter solrDispatchFilter = (SolrDispatchFilter) jetty.getDispatchFilter().getFilter();
if (solrDispatchFilter != null) {
CoreContainer cores = solrDispatchFilter.getCores();
if (cores != null) {
long sessionId = cores.getZkController().getZkClient().getSolrZooKeeper().getSessionId();
zkServer.expire(sessionId);
}
}
}
public void expireRandomSession() throws KeeperException, InterruptedException {
String sliceName = getRandomSlice();
JettySolrRunner jetty = getRandomJetty(sliceName, aggressivelyKillLeaders);
if (jetty != null) {
expireSession(jetty);
expires.incrementAndGet();
}
}
public void randomConnectionLoss() throws KeeperException, InterruptedException {
String sliceName = getRandomSlice();
JettySolrRunner jetty = getRandomJetty(sliceName, aggressivelyKillLeaders);
if (jetty != null) {
causeConnectionLoss(jetty);
connloss.incrementAndGet();
}
}
private void causeConnectionLoss(JettySolrRunner jetty) {
SolrDispatchFilter solrDispatchFilter = (SolrDispatchFilter) jetty
.getDispatchFilter().getFilter();
if (solrDispatchFilter != null) {
CoreContainer cores = solrDispatchFilter.getCores();
if (cores != null) {
SolrZkClient zkClient = cores.getZkController().getZkClient();
// must be at least double tick time...
zkClient.getSolrZooKeeper().pauseCnxn(ZkTestServer.TICK_TIME * 2);
}
}
}
public JettySolrRunner stopShard(String slice, int index) throws Exception {
JettySolrRunner jetty = shardToJetty.get(slice).get(index).jetty;
stopJetty(jetty);
return jetty;
}
public void stopJetty(JettySolrRunner jetty) throws Exception {
stop(jetty);
stops.incrementAndGet();
}
public void killJetty(JettySolrRunner jetty) throws Exception {
kill(jetty);
stops.incrementAndGet();
}
public static void stop(JettySolrRunner jetty) throws Exception {
// get a clean shutdown so that no dirs are left open...
FilterHolder fh = jetty.getDispatchFilter();
if (fh != null) {
SolrDispatchFilter sdf = (SolrDispatchFilter) fh.getFilter();
if (sdf != null) {
sdf.destroy();
}
}
jetty.stop();
if (!jetty.isStopped()) {
throw new RuntimeException("could not stop jetty");
}
}
public static void kill(JettySolrRunner jetty) throws Exception {
FilterHolder fh = jetty.getDispatchFilter();
SolrDispatchFilter sdf = null;
if (fh != null) {
sdf = (SolrDispatchFilter) fh.getFilter();
}
jetty.stop();
if (sdf != null) {
sdf.destroy();
}
if (!jetty.isStopped()) {
throw new RuntimeException("could not kill jetty");
}
}
public void stopShard(String slice) throws Exception {
List<CloudJettyRunner> jetties = shardToJetty.get(slice);
for (CloudJettyRunner jetty : jetties) {
stopJetty(jetty.jetty);
}
}
public void stopShardExcept(String slice, String shardName) throws Exception {
List<CloudJettyRunner> jetties = shardToJetty.get(slice);
for (CloudJettyRunner jetty : jetties) {
if (!jetty.nodeName.equals(shardName)) {
stopJetty(jetty.jetty);
}
}
}
public JettySolrRunner getShard(String slice, int index) throws Exception {
JettySolrRunner jetty = shardToJetty.get(slice).get(index).jetty;
return jetty;
}
public JettySolrRunner stopRandomShard() throws Exception {
String sliceName = getRandomSlice();
return stopRandomShard(sliceName);
}
public JettySolrRunner stopRandomShard(String slice) throws Exception {
JettySolrRunner jetty = getRandomJetty(slice, aggressivelyKillLeaders);
if (jetty != null) {
stopJetty(jetty);
}
return jetty;
}
public JettySolrRunner killRandomShard() throws Exception {
// add all the shards to a list
String sliceName = getRandomSlice();
return killRandomShard(sliceName);
}
private String getRandomSlice() {
Map<String,Slice> slices = zkStateReader.getCloudState().getSlices(collection);
List<String> sliceKeyList = new ArrayList<String>(slices.size());
sliceKeyList.addAll(slices.keySet());
String sliceName = sliceKeyList.get(random.nextInt(sliceKeyList.size()));
return sliceName;
}
public JettySolrRunner killRandomShard(String slice) throws Exception {
JettySolrRunner jetty = getRandomJetty(slice, aggressivelyKillLeaders);
if (jetty != null) {
killJetty(jetty);
}
return jetty;
}
public JettySolrRunner getRandomJetty(String slice, boolean aggressivelyKillLeaders) throws KeeperException, InterruptedException {
// get latest cloud state
zkStateReader.updateCloudState(true);
Slice theShards = zkStateReader.getCloudState().getSlices(collection)
.get(slice);
int numRunning = 0;
int numRecovering = 0;
int numActive = 0;
for (CloudJettyRunner cloudJetty : shardToJetty.get(slice)) {
boolean running = true;
ZkNodeProps props = theShards.getShards().get(cloudJetty.coreNodeName);
if (props == null) {
throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getShards().keySet());
}
String state = props.get(ZkStateReader.STATE_PROP);
String nodeName = props.get(ZkStateReader.NODE_NAME_PROP);
if (!cloudJetty.jetty.isRunning()
|| !state.equals(ZkStateReader.ACTIVE)
|| !zkStateReader.getCloudState().liveNodesContain(nodeName)) {
running = false;
}
if (cloudJetty.jetty.isRunning()
&& state.equals(ZkStateReader.RECOVERING)
&& zkStateReader.getCloudState().liveNodesContain(nodeName)) {
numRecovering++;
}
if (cloudJetty.jetty.isRunning()
&& state.equals(ZkStateReader.ACTIVE)
&& zkStateReader.getCloudState().liveNodesContain(nodeName)) {
numActive++;
}
if (running) {
numRunning++;
}
}
if (numActive < 2) {
// we cannot kill anyone
return null;
}
int chance = random.nextInt(10);
JettySolrRunner jetty;
if (chance <= 5 && aggressivelyKillLeaders) {
// if killLeader, really aggressively go after leaders
Collection<CloudJettyRunner> leaders = shardToLeaderJetty.values();
List<CloudJettyRunner> leadersList = new ArrayList<CloudJettyRunner>(leaders.size());
leadersList.addAll(leaders);
int index = random.nextInt(leadersList.size());
jetty = leadersList.get(index).jetty;
} else {
// get random shard
List<CloudJettyRunner> jetties = shardToJetty.get(slice);
int index = random.nextInt(jetties.size());
jetty = jetties.get(index).jetty;
ZkNodeProps leader = zkStateReader.getLeaderProps(collection, slice);
boolean isLeader = leader.get(ZkStateReader.NODE_NAME_PROP).equals(jetties.get(index).nodeName);
if (!aggressivelyKillLeaders && isLeader) {
// we don't kill leaders...
return null;
}
}
if (jetty.getLocalPort() == -1) {
// we can't kill the dead
return null;
}
//System.out.println("num active:" + numActive + " for " + slice + " sac:" + jetty.getLocalPort());
return jetty;
}
public SolrServer getRandomClient(String slice) throws KeeperException, InterruptedException {
// get latest cloud state
zkStateReader.updateCloudState(true);
// get random shard
List<SolrServer> clients = shardToClient.get(slice);
int index = random.nextInt(clients.size() - 1);
SolrServer client = clients.get(index);
return client;
}
// synchronously starts and stops shards randomly, unless there is only one
// active shard up for a slice or if there is one active and others recovering
public void startTheMonkey(boolean killLeaders, final int roundPause) {
this.aggressivelyKillLeaders = killLeaders;
startTime = System.currentTimeMillis();
// TODO: when kill leaders is on, lets kill a higher percentage of leaders
stop = false;
new Thread() {
private List<JettySolrRunner> deadPool = new ArrayList<JettySolrRunner>();
@Override
public void run() {
while (!stop) {
try {
Thread.sleep(roundPause);
if (random.nextBoolean()) {
if (!deadPool.isEmpty()) {
int index = random.nextInt(deadPool.size());
JettySolrRunner jetty = deadPool.get(index);
try {
jetty.start();
} catch (BindException e) {
jetty.stop();
sleep(2000);
try {
jetty.start();
} catch (BindException e2) {
jetty.stop();
sleep(5000);
try {
jetty.start();
} catch (BindException e3) {
// we coud not get the port
jetty.stop();
continue;
}
}
}
//System.out.println("started on port:" + jetty.getLocalPort());
deadPool.remove(index);
starts.incrementAndGet();
continue;
}
}
int rnd = random.nextInt(10);
if (expireSessions && rnd < EXPIRE_PERCENT) {
expireRandomSession();
}
if (causeConnectionLoss && rnd < CONLOSS_PERCENT) {
randomConnectionLoss();
randomConnectionLoss();
}
JettySolrRunner jetty;
if (random.nextBoolean()) {
jetty = stopRandomShard();
} else {
jetty = killRandomShard();
}
if (jetty == null) {
// we cannot kill
} else {
deadPool.add(jetty);
}
} catch (InterruptedException e) {
//
} catch (Exception e) {
// TODO Auto-generated catch block
e.printStackTrace();
}
}
System.out.println("I ran for " + (System.currentTimeMillis() - startTime)/1000.0f + "sec. I stopped " + stops + " and I started " + starts
+ ". I also expired " + expires.get() + " and caused " + connloss
+ " connection losses");
}
}.start();
}
public void stopTheMonkey() {
stop = true;
}
public int getStarts() {
return starts.get();
}
}

View File

@ -0,0 +1,299 @@
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.ConnectException;
import java.net.MalformedURLException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.commons.httpclient.HttpClient;
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.impl.StreamingUpdateSolrServer;
import org.apache.solr.common.SolrInputDocument;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
public class ChaosMonkeyNothingIsSafeTest extends FullSolrCloudTest {
@BeforeClass
public static void beforeSuperClass() throws Exception {
}
@AfterClass
public static void afterSuperClass() throws Exception {
}
@Before
@Override
public void setUp() throws Exception {
super.setUp();
// TODO use @Noisy annotation as we expect lots of exceptions
ignoreException(".*");
System.setProperty("numShards", Integer.toString(sliceCount));
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
resetExceptionIgnores();
}
public ChaosMonkeyNothingIsSafeTest() {
super();
shardCount = atLeast(3);
sliceCount = 2;
}
@Override
public void doTest() throws Exception {
handle.clear();
handle.put("QTime", SKIPVAL);
handle.put("timestamp", SKIPVAL);
// we cannot do delete by query
// as it's not supported for recovery
//del("*:*");
List<StopableIndexingThread> threads = new ArrayList<StopableIndexingThread>();
int threadCount = 1;
int i = 0;
for (i = 0; i < threadCount; i++) {
StopableIndexingThread indexThread = new StopableIndexingThread(i * 50000, true);
threads.add(indexThread);
indexThread.start();
}
FullThrottleStopableIndexingThread ftIndexThread = new FullThrottleStopableIndexingThread(
clients, i * 50000, true);
threads.add(ftIndexThread);
ftIndexThread.start();
chaosMonkey.startTheMonkey(true, 1500);
try {
Thread.sleep(atLeast(6000));
} finally {
chaosMonkey.stopTheMonkey();
}
for (StopableIndexingThread indexThread : threads) {
indexThread.safeStop();
}
// wait for stop...
for (StopableIndexingThread indexThread : threads) {
indexThread.join();
}
// fails will happen...
// for (StopableIndexingThread indexThread : threads) {
// assertEquals(0, indexThread.getFails());
// }
// try and wait for any replications and what not to finish...
Thread.sleep(2000);
// wait until there are no recoveries...
waitForThingsToLevelOut();
// make sure we again have leaders for each shard
for (int j = 1; j < sliceCount; j++) {
zkStateReader.getLeaderProps(DEFAULT_COLLECTION, "shard" + j, 10000);
}
commit();
// TODO: assert we didnt kill everyone
zkStateReader.updateCloudState(true);
assertTrue(zkStateReader.getCloudState().getLiveNodes().size() > 0);
checkShardConsistency(false, false);
// ensure we have added more than 0 docs
long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
assertTrue(cloudClientDocs > 0);
if (VERBOSE) System.out.println("control docs:" + controlClient.query(new SolrQuery("*:*")).getResults().getNumFound() + "\n\n");
}
private void waitForThingsToLevelOut() throws KeeperException,
InterruptedException, Exception, IOException, URISyntaxException {
int cnt = 0;
boolean retry = false;
do {
waitForRecoveriesToFinish(VERBOSE);
commit();
updateMappingsFromZk(jettys, clients);
Set<String> theShards = shardToClient.keySet();
String failMessage = null;
for (String shard : theShards) {
failMessage = checkShardConsistency(shard, false);
}
if (failMessage != null) {
retry = true;
}
cnt++;
if (cnt > 10) break;
Thread.sleep(4000);
} while (retry);
}
// skip the randoms - they can deadlock...
protected void indexr(Object... fields) throws Exception {
SolrInputDocument doc = getDoc(fields);
indexDoc(doc);
}
private SolrInputDocument getDoc(Object... fields) {
SolrInputDocument doc = new SolrInputDocument();
addFields(doc, fields);
addFields(doc, "rnd_b", true);
return doc;
}
class FullThrottleStopableIndexingThread extends StopableIndexingThread {
MultiThreadedHttpConnectionManager cm = new MultiThreadedHttpConnectionManager();
private HttpClient httpClient = new HttpClient(cm) ;
private volatile boolean stop = false;
int clientIndex = 0;
private StreamingUpdateSolrServer suss;
private List<SolrServer> clients;
public FullThrottleStopableIndexingThread(List<SolrServer> clients,
int startI, boolean doDeletes) throws MalformedURLException {
super(startI, doDeletes);
setName("FullThrottleStopableIndexingThread");
setDaemon(true);
this.clients = clients;
suss = new StreamingUpdateSolrServer(
((CommonsHttpSolrServer) clients.get(0)).getBaseURL(), httpClient, 8,
2) {
public void handleError(Throwable ex) {
log.warn("suss error", ex);
}
};
}
@Override
public void run() {
int i = startI;
int numDeletes = 0;
int numAdds = 0;
while (true && !stop) {
++i;
if (doDeletes && random.nextBoolean() && deletes.size() > 0) {
Integer delete = deletes.remove(0);
try {
numDeletes++;
suss.deleteById(Integer.toString(delete));
} catch (Exception e) {
changeUrlOnError(e);
//System.err.println("REQUEST FAILED:");
//e.printStackTrace();
fails.incrementAndGet();
}
}
try {
numAdds++;
if (numAdds > 4000)
continue;
SolrInputDocument doc = getDoc(
id,
i,
i1,
50,
tlong,
50,
t1,
"Saxon heptarchies that used to rip around so in old times and raise Cain. My, you ought to seen old Henry the Eight when he was in bloom. He WAS a blossom. He used to marry a new wife every day, and chop off her head next morning. And he would do it just as indifferent as if ");
suss.add(doc);
} catch (Exception e) {
changeUrlOnError(e);
//System.err.println("REQUEST FAILED:");
//e.printStackTrace();
fails.incrementAndGet();
}
if (doDeletes && random.nextBoolean()) {
deletes.add(i);
}
}
System.err.println("FT added docs:" + numAdds + " with " + fails + " fails" + " deletes:" + numDeletes);
}
private void changeUrlOnError(Exception e) {
if (e instanceof ConnectException) {
clientIndex++;
if (clientIndex > clients.size() - 1) {
clientIndex = 0;
}
try {
suss.shutdownNow();
suss = new StreamingUpdateSolrServer(
((CommonsHttpSolrServer) clients.get(clientIndex)).getBaseURL(),
httpClient, 30, 3) {
public void handleError(Throwable ex) {
log.warn("suss error", ex);
}
};
} catch (MalformedURLException e1) {
e1.printStackTrace();
}
}
}
public void safeStop() {
stop = true;
suss.shutdownNow();
cm.shutdown();
}
public int getFails() {
return fails.get();
}
};
}

View File

@ -0,0 +1,156 @@
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.URISyntaxException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.common.SolrInputDocument;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
public class ChaosMonkeySafeLeaderTest extends FullSolrCloudTest {
@BeforeClass
public static void beforeSuperClass() throws Exception {
}
@AfterClass
public static void afterSuperClass() throws Exception {
}
@Before
@Override
public void setUp() throws Exception {
super.setUp();
// we expect this time of exception as shards go up and down...
ignoreException(".*");
// sometimes we cannot get the same port
ignoreException("java\\.net\\.BindException: Address already in use");
System.setProperty("numShards", Integer.toString(sliceCount));
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
resetExceptionIgnores();
}
public ChaosMonkeySafeLeaderTest() {
super();
sliceCount = atLeast(2);
shardCount = atLeast(sliceCount);
}
@Override
public void doTest() throws Exception {
handle.clear();
handle.put("QTime", SKIPVAL);
handle.put("timestamp", SKIPVAL);
// we cannot do delete by query
// as it's not supported for recovery
//del("*:*");
List<StopableIndexingThread> threads = new ArrayList<StopableIndexingThread>();
int threadCount = 2;
for (int i = 0; i < threadCount; i++) {
StopableIndexingThread indexThread = new StopableIndexingThread(i * 50000, true);
threads.add(indexThread);
indexThread.start();
}
chaosMonkey.startTheMonkey(false, 500);
Thread.sleep(atLeast(8000));
chaosMonkey.stopTheMonkey();
for (StopableIndexingThread indexThread : threads) {
indexThread.safeStop();
}
// wait for stop...
for (StopableIndexingThread indexThread : threads) {
indexThread.join();
}
for (StopableIndexingThread indexThread : threads) {
assertEquals(0, indexThread.getFails());
}
// try and wait for any replications and what not to finish...
waitForThingsToLevelOut();
checkShardConsistency(true, false);
if (VERBOSE) System.out.println("control docs:" + controlClient.query(new SolrQuery("*:*")).getResults().getNumFound() + "\n\n");
}
private void waitForThingsToLevelOut() throws KeeperException,
InterruptedException, Exception, IOException, URISyntaxException {
int cnt = 0;
boolean retry = false;
do {
waitForRecoveriesToFinish(false);
commit();
updateMappingsFromZk(jettys, clients);
Set<String> theShards = shardToClient.keySet();
String failMessage = null;
for (String shard : theShards) {
failMessage = checkShardConsistency(shard, false);
}
if (failMessage != null) {
retry = true;
} else {
retry = false;
}
cnt++;
if (cnt > 10) break;
Thread.sleep(2000);
} while (retry);
}
// skip the randoms - they can deadlock...
protected void indexr(Object... fields) throws Exception {
SolrInputDocument doc = new SolrInputDocument();
addFields(doc, fields);
addFields(doc, "rnd_b", true);
indexDoc(doc);
}
}

View File

@ -0,0 +1,78 @@
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.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.junit.Test;
public class CloudStateTest extends SolrTestCaseJ4 {
@Test
public void testStoreAndRead() throws Exception {
Map<String,Map<String,Slice>> collectionStates = new HashMap<String,Map<String,Slice>>();
Set<String> liveNodes = new HashSet<String>();
liveNodes.add("node1");
liveNodes.add("node2");
Map<String,Slice> slices = new HashMap<String,Slice>();
Map<String,ZkNodeProps> sliceToProps = new HashMap<String,ZkNodeProps>();
Map<String,String> props = new HashMap<String,String>();
props.put("prop1", "value");
props.put("prop2", "value2");
ZkNodeProps zkNodeProps = new ZkNodeProps(props);
sliceToProps.put("node1", zkNodeProps);
Slice slice = new Slice("shard1", sliceToProps);
slices.put("shard1", slice);
Slice slice2 = new Slice("shard2", sliceToProps);
slices.put("shard2", slice2);
collectionStates.put("collection1", slices);
collectionStates.put("collection2", slices);
CloudState cloudState = new CloudState(liveNodes, collectionStates);
byte[] bytes = ZkStateReader.toJSON(cloudState);
CloudState loadedCloudState = CloudState.load(bytes, liveNodes);
assertEquals("Provided liveNodes not used properly", 2, loadedCloudState
.getLiveNodes().size());
assertEquals("No collections found", 2, loadedCloudState.getCollections().size());
assertEquals("Poperties not copied properly", zkNodeProps.get("prop1"), loadedCloudState.getSlice("collection1", "shard1").getShards().get("node1").get("prop1"));
assertEquals("Poperties not copied properly", zkNodeProps.get("prop2"), loadedCloudState.getSlice("collection1", "shard1").getShards().get("node1").get("prop2"));
loadedCloudState = CloudState.load(new byte[0], liveNodes);
assertEquals("Provided liveNodes not used properly", 2, loadedCloudState
.getLiveNodes().size());
assertEquals("Should not have collections", 0, loadedCloudState.getCollections().size());
loadedCloudState = CloudState.load((byte[])null, liveNodes);
assertEquals("Provided liveNodes not used properly", 2, loadedCloudState
.getLiveNodes().size());
assertEquals("Should not have collections", 0, loadedCloudState.getCollections().size());
}
}

View File

@ -18,7 +18,7 @@ package org.apache.solr.cloud;
*/
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
@ -31,35 +31,20 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreContainer.Initializer;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* TODO: look at hostPort used below
*/
public class CloudStateUpdateTest extends SolrTestCaseJ4 {
protected static Logger log = LoggerFactory
.getLogger(AbstractZkTestCase.class);
private static final boolean VERBOSE = false;
private static final String URL1 = "http://localhost:3133/solr/core0";
private static final String URL3 = "http://localhost:3133/solr/core1";
private static final String URL2 = "http://localhost:3123/solr/core1";
private static final String URL4 = "http://localhost:3123/solr/core4";
private static final String SHARD4 = "localhost:3123_solr_core4";
private static final String SHARD3 = "localhost:3123_solr_core3";
private static final String SHARD2 = "localhost:3123_solr_core2";
private static final String SHARD1 = "localhost:3123_solr_core1";
private static final int TIMEOUT = 10000;
protected ZkTestServer zkServer;
protected String zkDir;
@ -82,6 +67,14 @@ public class CloudStateUpdateTest extends SolrTestCaseJ4 {
@BeforeClass
public static void beforeClass() throws Exception {
System.setProperty("solrcloud.skip.autorecovery", "true");
}
@AfterClass
public static void afterClass() throws InterruptedException {
System.clearProperty("solrcloud.skip.autorecovery");
// wait just a bit for any zk client threads to outlast timeout
Thread.sleep(2000);
}
@Override
@ -115,6 +108,7 @@ public class CloudStateUpdateTest extends SolrTestCaseJ4 {
System.setProperty("solr.test.sys.prop1", "propone");
System.setProperty("solr.test.sys.prop2", "proptwo");
System.setProperty("solr.solr.home", TEST_HOME());
System.setProperty("hostPort", "1661");
CoreContainer.Initializer init1 = new CoreContainer.Initializer();
System.setProperty("solr.data.dir", CloudStateUpdateTest.this.dataDir1.getAbsolutePath());
@ -133,85 +127,28 @@ public class CloudStateUpdateTest extends SolrTestCaseJ4 {
System.setProperty("solr.data.dir", CloudStateUpdateTest.this.dataDir3.getAbsolutePath());
container3 = init3.initialize();
System.clearProperty("hostPort");
System.clearProperty("solr.solr.home");
log.info("####SETUP_END " + getName());
}
@Test
public void testIncrementalUpdate() throws Exception {
System.setProperty("CLOUD_UPDATE_DELAY", "1");
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = null;
SolrZkClient zkClient = null;
ZkController zkController = null;
server = new ZkTestServer(zkDir);
server.run();
try {
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
String shardsPath1 = "/collections/collection1/shards/shardid1";
String shardsPath2 = "/collections/collection1/shards/shardid2";
zkClient.makePath(shardsPath1);
zkClient.makePath(shardsPath2);
addShardToZk(zkClient, shardsPath1, SHARD1, URL1);
addShardToZk(zkClient, shardsPath1, SHARD2, URL2);
addShardToZk(zkClient, shardsPath2, SHARD3, URL3);
removeShardFromZk(server.getZkAddress(), zkClient, shardsPath1);
zkController = new ZkController(server.getZkAddress(), TIMEOUT, 1000,
"localhost", "8983", "solr");
zkController.getZkStateReader().updateCloudState(true);
CloudState cloudInfo = zkController.getCloudState();
Map<String,Slice> slices = cloudInfo.getSlices("collection1");
assertFalse(slices.containsKey("shardid1"));
zkClient.makePath(shardsPath1);
addShardToZk(zkClient, shardsPath1, SHARD1, URL1);
zkController.getZkStateReader().updateCloudState(true);
cloudInfo = zkController.getCloudState();
slices = cloudInfo.getSlices("collection1");
assertTrue(slices.containsKey("shardid1"));
updateUrl(zkClient, shardsPath1, SHARD1, "fake");
addShardToZk(zkClient, shardsPath2, SHARD4, URL4);
zkController.getZkStateReader().updateCloudState(true);
cloudInfo = zkController.getCloudState();
String url = cloudInfo.getSlices("collection1").get("shardid1").getShards().get(SHARD1).get("url");
// because of incremental update, we don't expect to find the new 'fake'
// url - instead we should still
// be using the original url - the correct way to update this would be to
// remove the whole node and readd it
assertEquals(URL1, url);
} finally {
server.shutdown();
zkClient.close();
zkController.close();
}
}
@Test
public void testCoreRegistration() throws Exception {
System.setProperty("CLOUD_UPDATE_DELAY", "1");
System.setProperty("solrcloud.update.delay", "1");
ZkNodeProps props2 = new ZkNodeProps();
Map<String,String> props2 = new HashMap<String,String>();
props2.put("configName", "conf1");
ZkNodeProps zkProps2 = new ZkNodeProps(props2);
SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
zkClient.makePath("/collections/testcore", props2.store(), CreateMode.PERSISTENT);
zkClient.makePath("/collections/testcore/shards", CreateMode.PERSISTENT);
SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(),
AbstractZkTestCase.TIMEOUT);
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/testcore",
ZkStateReader.toJSON(zkProps2), CreateMode.PERSISTENT, true);
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/testcore/shards",
CreateMode.PERSISTENT, true);
zkClient.close();
CoreDescriptor dcore = new CoreDescriptor(container1, "testcore",
@ -235,17 +172,18 @@ public class CloudStateUpdateTest extends SolrTestCaseJ4 {
cloudState2 = zkController2.getCloudState();
slices = cloudState2.getSlices("testcore");
if (slices != null && slices.containsKey(host + ":1661_solr_testcore")) {
if (slices != null && slices.containsKey("shard1")
&& slices.get("shard1").getShards().size() > 0) {
break;
}
Thread.sleep(500);
}
assertNotNull(slices);
assertTrue(slices.containsKey(host + ":1661_solr_testcore"));
assertTrue(slices.containsKey("shard1"));
Slice slice = slices.get(host + ":1661_solr_testcore");
assertEquals(host + ":1661_solr_testcore", slice.getName());
Slice slice = slices.get("shard1");
assertEquals("shard1", slice.getName());
Map<String,ZkNodeProps> shards = slice.getShards();
@ -255,9 +193,9 @@ public class CloudStateUpdateTest extends SolrTestCaseJ4 {
assertNotNull(zkProps);
assertEquals(host + ":1661_solr", zkProps.get("node_name"));
assertEquals(host + ":1661_solr", zkProps.get(ZkStateReader.NODE_NAME_PROP));
assertEquals("http://" + host + ":1661/solr/testcore", zkProps.get("url"));
assertEquals("http://" + host + ":1661/solr", zkProps.get(ZkStateReader.BASE_URL_PROP));
Set<String> liveNodes = cloudState2.getLiveNodes();
assertNotNull(liveNodes);
@ -305,44 +243,15 @@ public class CloudStateUpdateTest extends SolrTestCaseJ4 {
}
container1.shutdown();
container2.shutdown();
if (!container3.isShutDown()) {
container3.shutdown();
}
zkServer.shutdown();
super.tearDown();
System.clearProperty("zkClientTimeout");
System.clearProperty("zkHost");
System.clearProperty("hostPort");
System.clearProperty("CLOUD_UPDATE_DELAY");
}
private void addShardToZk(SolrZkClient zkClient, String shardsPath,
String zkNodeName, String url) throws IOException,
KeeperException, InterruptedException {
ZkNodeProps props = new ZkNodeProps();
props.put(ZkStateReader.URL_PROP, url);
props.put(ZkStateReader.NODE_NAME, zkNodeName);
byte[] bytes = props.store();
zkClient
.create(shardsPath + "/" + zkNodeName, bytes, CreateMode.PERSISTENT);
}
private void updateUrl(SolrZkClient zkClient, String shardsPath,
String zkNodeName, String url) throws IOException,
KeeperException, InterruptedException {
ZkNodeProps props = new ZkNodeProps();
props.put(ZkStateReader.URL_PROP, url);
props.put(ZkStateReader.NODE_NAME, zkNodeName);
byte[] bytes = props.store();
zkClient
.setData(shardsPath + "/" + zkNodeName, bytes);
}
private void removeShardFromZk(String zkHost, SolrZkClient zkClient, String shardsPath) throws Exception {
AbstractZkTestCase.tryCleanPath(zkHost, shardsPath);
System.clearProperty("solrcloud.update.delay");
}
private void printLayout(String zkHost) throws Exception {

View File

@ -0,0 +1,305 @@
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.concurrent.TimeoutException;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.impl.StreamingUpdateSolrServer;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.SolrZkClient;
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.update.VersionInfo;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.junit.BeforeClass;
/**
* Super basic testing, no shard restarting or anything.
*/
public class FullSolrCloudDistribCmdsTest extends FullSolrCloudTest {
@BeforeClass
public static void beforeSuperClass() throws Exception {
}
public FullSolrCloudDistribCmdsTest() {
super();
shardCount = 4;
sliceCount = 2;
}
@Override
public void doTest() throws Exception {
handle.clear();
handle.put("QTime", SKIPVAL);
handle.put("timestamp", SKIPVAL);
waitForRecoveriesToFinish(false);
// add a doc, update it, and delete it
QueryResponse results;
UpdateRequest uReq;
long docId = addUpdateDelete();
// add 2 docs in a request
SolrInputDocument doc1;
SolrInputDocument doc2;
docId = addTwoDocsInOneRequest(docId);
// two deletes
uReq = new UpdateRequest();
uReq.deleteById(Long.toString(docId-1));
uReq.deleteById(Long.toString(docId-2)).process(cloudClient);
controlClient.deleteById(Long.toString(docId-1));
controlClient.deleteById(Long.toString(docId-2));
commit();
results = query(cloudClient);
assertEquals(0, results.getResults().getNumFound());
results = query(controlClient);
assertEquals(0, results.getResults().getNumFound());
// add two docs together, a 3rd doc and a delete
indexr("id", docId++, t1, "originalcontent");
uReq = new UpdateRequest();
doc1 = new SolrInputDocument();
addFields(doc1, "id", docId++);
uReq.add(doc1);
doc2 = new SolrInputDocument();
addFields(doc2, "id", docId++);
uReq.add(doc2);
uReq.process(cloudClient);
uReq.process(controlClient);
uReq = new UpdateRequest();
uReq.deleteById(Long.toString(docId - 2)).process(cloudClient);
controlClient.deleteById(Long.toString(docId - 2));
commit();
assertDocCounts(VERBOSE);
checkShardConsistency();
results = query(controlClient);
assertEquals(2, results.getResults().getNumFound());
results = query(cloudClient);
assertEquals(2, results.getResults().getNumFound());
testIndexingWithSuss();
// TODO: testOptimisticUpdate(results);
testDeleteByQueryDistrib();
testThatCantForwardToLeaderFails();
}
private void testThatCantForwardToLeaderFails() throws InterruptedException,
Exception, TimeoutException, IOException, KeeperException {
ZkNodeProps props = zkStateReader.getLeaderProps(DEFAULT_COLLECTION, "shard1");
chaosMonkey.stopShard("shard1");
// fake that the leader is still advertised
String leaderPath = ZkStateReader.getShardLeadersPath(DEFAULT_COLLECTION, "shard1");
SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), 10000);
int fails = 0;
try {
zkClient.makePath(leaderPath, ZkStateReader.toJSON(props),
CreateMode.EPHEMERAL, true);
for (int i = 200; i < 210; i++) {
try {
index_specific(cloudClient, id, i);
} catch (SolrException e) {
// expected
fails++;
break;
} catch (SolrServerException e) {
// expected
fails++;
break;
}
}
} finally {
zkClient.close();
}
assertTrue("A whole shard is down - some of these should fail", fails > 0);
}
private long addTwoDocsInOneRequest(long docId) throws SolrServerException,
IOException, Exception {
QueryResponse results;
UpdateRequest uReq;
uReq = new UpdateRequest();
//uReq.setParam(UpdateParams.UPDATE_CHAIN, DISTRIB_UPDATE_CHAIN);
SolrInputDocument doc1 = new SolrInputDocument();
addFields(doc1, "id", docId++);
uReq.add(doc1);
SolrInputDocument doc2 = new SolrInputDocument();
addFields(doc2, "id", docId++);
uReq.add(doc2);
uReq.process(cloudClient);
uReq.process(controlClient);
commit();
checkShardConsistency();
assertDocCounts(VERBOSE);
results = query(cloudClient);
assertEquals(2, results.getResults().getNumFound());
return docId;
}
private long addUpdateDelete() throws Exception, SolrServerException,
IOException {
long docId = 99999999L;
indexr("id", docId, t1, "originalcontent");
commit();
ModifiableSolrParams params = new ModifiableSolrParams();
params.add("q", t1 + ":originalcontent");
QueryResponse results = clients.get(0).query(params);
assertEquals(1, results.getResults().getNumFound());
// update doc
indexr("id", docId, t1, "updatedcontent");
commit();
assertDocCounts(VERBOSE);
results = clients.get(0).query(params);
assertEquals(0, results.getResults().getNumFound());
params.set("q", t1 + ":updatedcontent");
results = clients.get(0).query(params);
assertEquals(1, results.getResults().getNumFound());
UpdateRequest uReq = new UpdateRequest();
//uReq.setParam(UpdateParams.UPDATE_CHAIN, DISTRIB_UPDATE_CHAIN);
uReq.deleteById(Long.toString(docId)).process(clients.get(0));
commit();
results = clients.get(0).query(params);
assertEquals(0, results.getResults().getNumFound());
return docId;
}
private void testDeleteByQueryDistrib() throws Exception, SolrServerException {
del("*:*");
commit();
assertEquals(0, query(cloudClient).getResults().getNumFound());
}
private void testIndexingWithSuss() throws MalformedURLException, Exception {
StreamingUpdateSolrServer suss = new StreamingUpdateSolrServer(
((CommonsHttpSolrServer) clients.get(0)).getBaseURL(), 3, 1);
for (int i=100; i<150; i++) {
index_specific(suss, id, i);
}
suss.blockUntilFinished();
commit();
checkShardConsistency();
}
private void testOptimisticUpdate(QueryResponse results) throws Exception {
SolrDocument doc = results.getResults().get(0);
System.out.println("version:" + doc.getFieldValue(VersionInfo.VERSION_FIELD));
Long version = (Long) doc.getFieldValue(VersionInfo.VERSION_FIELD);
Integer theDoc = (Integer) doc.getFieldValue("id");
UpdateRequest uReq = new UpdateRequest();
SolrInputDocument doc1 = new SolrInputDocument();
uReq.setParams(new ModifiableSolrParams());
uReq.getParams().set(DistributedUpdateProcessor.VERSION_FIELD, Long.toString(version));
addFields(doc1, "id", theDoc, t1, "theupdatestuff");
uReq.add(doc1);
uReq.process(cloudClient);
uReq.process(controlClient);
commit();
// updating the old version should fail...
SolrInputDocument doc2 = new SolrInputDocument();
uReq = new UpdateRequest();
uReq.setParams(new ModifiableSolrParams());
uReq.getParams().set(DistributedUpdateProcessor.VERSION_FIELD, Long.toString(version));
addFields(doc2, "id", theDoc, t1, "thenewupdatestuff");
uReq.add(doc2);
uReq.process(cloudClient);
uReq.process(controlClient);
commit();
ModifiableSolrParams params = new ModifiableSolrParams();
params.add("q", t1 + ":thenewupdatestuff");
QueryResponse res = clients.get(0).query(params);
assertEquals(0, res.getResults().getNumFound());
params = new ModifiableSolrParams();
params.add("q", t1 + ":theupdatestuff");
res = clients.get(0).query(params);
assertEquals(1, res.getResults().getNumFound());
}
private QueryResponse query(SolrServer server) throws SolrServerException {
SolrQuery query = new SolrQuery("*:*");
return server.query(query);
}
@Override
public void tearDown() throws Exception {
super.tearDown();
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,287 @@
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.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import javax.xml.parsers.ParserConfigurationException;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.SolrZkClient;
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.CoreContainer.Initializer;
import org.apache.solr.core.SolrConfig;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.xml.sax.SAXException;
public class LeaderElectionIntegrationTest extends SolrTestCaseJ4 {
protected static Logger log = LoggerFactory
.getLogger(AbstractZkTestCase.class);
private final static int NUM_SHARD_REPLICAS = 5;
private static final boolean VERBOSE = false;
private static final Pattern HOST = Pattern
.compile(".*?\\:(\\d\\d\\d\\d)_.*");
protected ZkTestServer zkServer;
protected String zkDir;
private Map<Integer,CoreContainer> containerMap = new HashMap<Integer,CoreContainer>();
private Map<String,Set<Integer>> shardPorts = new HashMap<String,Set<Integer>>();
private SolrZkClient zkClient;
private ZkStateReader reader;
@BeforeClass
public static void beforeClass() throws Exception {
System.setProperty("solrcloud.skip.autorecovery", "true");
}
@Override
public void setUp() throws Exception {
super.setUp();
createTempDir();
System.setProperty("zkClientTimeout", "3000");
zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
zkServer = new ZkTestServer(zkDir);
zkServer.run();
System.setProperty("zkHost", zkServer.getZkAddress());
AbstractZkTestCase.buildZooKeeper(zkServer.getZkHost(),
zkServer.getZkAddress(), "solrconfig.xml", "schema.xml");
log.info("####SETUP_START " + getName());
// set some system properties for use by tests
System.setProperty("solr.test.sys.prop1", "propone");
System.setProperty("solr.test.sys.prop2", "proptwo");
for (int i = 7000; i < 7000 + NUM_SHARD_REPLICAS; i++) {
setupContainer(i, "shard1");
}
setupContainer(3333, "shard2");
zkClient = new SolrZkClient(zkServer.getZkAddress(),
AbstractZkTestCase.TIMEOUT);
reader = new ZkStateReader(zkClient);
log.info("####SETUP_END " + getName());
}
private void setupContainer(int port, String shard) throws IOException,
ParserConfigurationException, SAXException {
File data = new File(dataDir + File.separator + "data_" + port);
data.mkdirs();
System.setProperty("hostPort", Integer.toString(port));
System.setProperty("shard", shard);
Initializer init = new CoreContainer.Initializer();
System.setProperty("solr.data.dir", data.getAbsolutePath());
System.setProperty("solr.solr.home", TEST_HOME());
Set<Integer> ports = shardPorts.get(shard);
if (ports == null) {
ports = new HashSet<Integer>();
shardPorts.put(shard, ports);
}
ports.add(port);
CoreContainer container = init.initialize();
containerMap.put(port, container);
System.clearProperty("solr.solr.home");
System.clearProperty("hostPort");
}
@Test
public void testSimpleSliceLeaderElection() throws Exception {
//printLayout(zkServer.getZkAddress());
for (int i = 0; i < 4; i++) {
// who is the leader?
String leader = getLeader();
Set<Integer> shard1Ports = shardPorts.get("shard1");
int leaderPort = getLeaderPort(leader);
assertTrue(shard1Ports.toString(), shard1Ports.contains(leaderPort));
shard1Ports.remove(leaderPort);
// kill the leader
if (VERBOSE) System.out.println("Killing " + leaderPort);
containerMap.get(leaderPort).shutdown();
//printLayout(zkServer.getZkAddress());
// poll until leader change is visible
for (int j = 0; j < 30; j++) {
String currentLeader = getLeader();
if(!leader.equals(currentLeader)) {
break;
}
Thread.sleep(100);
}
leader = getLeader();
int newLeaderPort = getLeaderPort(leader);
int retry = 0;
while (leaderPort == newLeaderPort) {
if (retry++ == 20) {
break;
}
Thread.sleep(1000);
}
if (leaderPort == newLeaderPort) {
fail("We didn't find a new leader! " + leaderPort + " was shutdown, but it's still showing as the leader");
}
assertTrue("Could not find leader " + newLeaderPort + " in " + shard1Ports, shard1Ports.contains(newLeaderPort));
}
}
@Test
public void testLeaderElectionAfterClientTimeout() throws Exception {
// TODO: work out the best timing here...
System.setProperty("zkClientTimeout", "500");
// timeout the leader
String leader = getLeader();
int leaderPort = getLeaderPort(leader);
containerMap.get(leaderPort).getZkController().getZkClient().getSolrZooKeeper().pauseCnxn(2000);
for (int i = 0; i < 60; i++) { // wait till leader is changed
if (leaderPort != getLeaderPort(getLeader())) {
break;
}
Thread.sleep(100);
}
if (VERBOSE) System.out.println("kill everyone");
// kill everyone but the first leader that should have reconnected by now
for (Map.Entry<Integer,CoreContainer> entry : containerMap.entrySet()) {
if (entry.getKey() != leaderPort) {
entry.getValue().shutdown();
}
}
for (int i = 0; i < 60; i++) { // wait till leader is changed
if (leaderPort == getLeaderPort(getLeader())) {
break;
}
Thread.sleep(100);
}
// the original leader should be leader again now - everyone else is down
// TODO: I saw this fail once...expected:<7000> but was:<7004>
assertEquals(leaderPort, getLeaderPort(getLeader()));
//printLayout(zkServer.getZkAddress());
//Thread.sleep(100000);
}
private String getLeader() throws InterruptedException {
String leader = null;
int tries = 30;
while (tries-- > 0) {
ZkNodeProps props;
try {
reader.updateCloudState(true);
props = reader.getLeaderProps("collection1", "shard1", 500);
leader = props.get(ZkStateReader.NODE_NAME_PROP);
if (leader != null) {
break;
}
} catch (KeeperException e) {
// ignore
}
Thread.sleep(200);
}
return leader;
}
private int getLeaderPort(String leader) {
Matcher m = HOST.matcher(leader);
int leaderPort = 0;
if (m.matches()) {
leaderPort = Integer.parseInt(m.group(1));
if (VERBOSE) System.out.println("The leader is:" + Integer.parseInt(m.group(1)));
} else {
throw new IllegalStateException();
}
return leaderPort;
}
@Override
public void tearDown() throws Exception {
if (VERBOSE) {
printLayout(zkServer.getZkHost());
}
if (zkClient != null) {
zkClient.close();
}
for (CoreContainer cc : containerMap.values()) {
if (!cc.isShutDown()) {
cc.shutdown();
}
}
zkServer.shutdown();
super.tearDown();
System.clearProperty("zkClientTimeout");
System.clearProperty("zkHost");
System.clearProperty("hostPort");
System.clearProperty("shard");
System.clearProperty("solrcloud.update.delay");
}
private void printLayout(String zkHost) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, AbstractZkTestCase.TIMEOUT);
zkClient.printLayoutToStdOut();
zkClient.close();
}
@AfterClass
public static void afterClass() throws InterruptedException {
System.clearProperty("solrcloud.skip.autorecovery");
// wait just a bit for any zk client threads to outlast timeout
Thread.sleep(2000);
}
}

View File

@ -0,0 +1,408 @@
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.File;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.solr.SolrTestCaseJ4;
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.SolrConfig;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
public class LeaderElectionTest extends SolrTestCaseJ4 {
static final int TIMEOUT = 30000;
private ZkTestServer server;
private SolrZkClient zkClient;
private ZkStateReader zkStateReader;
private Map<Integer,Thread> seqToThread;
private volatile boolean stopStress = false;
@BeforeClass
public static void beforeClass() throws Exception {
createTempDir();
}
@AfterClass
public static void afterClass() throws InterruptedException {
// wait just a bit for any zk client threads to outlast timeout
Thread.sleep(2000);
}
@Override
public void setUp() throws Exception {
super.setUp();
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
server = new ZkTestServer(zkDir);
server.setTheTickTime(1000);
server.run();
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
zkStateReader = new ZkStateReader(zkClient);
seqToThread = Collections.synchronizedMap(new HashMap<Integer,Thread>());
}
class ClientThread extends Thread {
SolrZkClient zkClient;
private int nodeNumber;
private volatile int seq = -1;
private volatile boolean stop;
private volatile boolean electionDone = false;
private final ZkNodeProps props;
private ZkStateReader zkStateReader;
public ClientThread(int nodeNumber) throws Exception {
super("Thread-" + nodeNumber);
this.zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
this.zkStateReader = new ZkStateReader(zkClient);
this.nodeNumber = nodeNumber;
props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, Integer.toString(nodeNumber), ZkStateReader.CORE_NAME_PROP, "");
}
@Override
public void run() {
LeaderElector elector = new LeaderElector(ClientThread.this.zkClient);
ElectionContext context = new ShardLeaderElectionContextBase(elector, "shard1",
"collection1", Integer.toString(nodeNumber), props, this.zkStateReader);
try {
elector.setup(context);
seq = elector.joinElection(context);
electionDone = true;
seqToThread.put(seq, this);
} catch (InterruptedException e) {
return;
} catch (Throwable e) {
//e.printStackTrace();
}
while (!stop) {
try {
Thread.sleep(100);
} catch (InterruptedException e) {
return;
}
}
}
public void close() throws InterruptedException {
if (!zkClient.isClosed()) {
zkClient.close();
}
this.stop = true;
}
public int getSeq() {
return seq;
}
public int getNodeNumber() {
return nodeNumber;
}
}
@Test
public void testBasic() throws Exception {
LeaderElector elector = new LeaderElector(zkClient);
ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
"http://127.0.0.1/solr/", ZkStateReader.CORE_NAME_PROP, "");
ElectionContext context = new ShardLeaderElectionContextBase(elector,
"shard2", "collection1", "dummynode1", props, zkStateReader);
elector.setup(context);
elector.joinElection(context);
assertEquals("http://127.0.0.1/solr/",
getLeaderUrl("collection1", "shard2"));
}
private String getLeaderUrl(final String collection, final String slice)
throws KeeperException, InterruptedException {
int iterCount = 30;
while (iterCount-- > 0)
try {
byte[] data = zkClient.getData(
ZkStateReader.getShardLeadersPath(collection, slice), null, null,
true);
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(
ZkNodeProps.load(data));
return leaderProps.getCoreUrl();
} catch (NoNodeException e) {
Thread.sleep(100);
}
throw new RuntimeException("Could not get leader props");
}
@Test
public void testElection() throws Exception {
List<ClientThread> threads = new ArrayList<ClientThread>();
for (int i = 0; i < 15; i++) {
ClientThread thread = new ClientThread(i);
threads.add(thread);
}
for (Thread thread : threads) {
thread.start();
}
while(true) { //wait for election to complete
int doneCount = 0;
for (ClientThread thread : threads) {
if(thread.electionDone) {
doneCount++;
}
}
if(doneCount==15) {
break;
}
Thread.sleep(100);
}
int leaderThread = getLeaderThread();
// whoever the leader is, should be the n_0 seq
assertEquals(0, threads.get(leaderThread).seq);
// kill n_0, 1, 3 and 4
((ClientThread) seqToThread.get(0)).close();
waitForLeader(threads, 1);
leaderThread = getLeaderThread();
// whoever the leader is, should be the n_1 seq
assertEquals(1, threads.get(leaderThread).seq);
((ClientThread) seqToThread.get(4)).close();
((ClientThread) seqToThread.get(1)).close();
((ClientThread) seqToThread.get(3)).close();
// whoever the leader is, should be the n_2 seq
waitForLeader(threads, 2);
leaderThread = getLeaderThread();
assertEquals(2, threads.get(leaderThread).seq);
// kill n_5, 2, 6, 7, and 8
((ClientThread) seqToThread.get(5)).close();
((ClientThread) seqToThread.get(2)).close();
((ClientThread) seqToThread.get(6)).close();
((ClientThread) seqToThread.get(7)).close();
((ClientThread) seqToThread.get(8)).close();
waitForLeader(threads, 9);
leaderThread = getLeaderThread();
// whoever the leader is, should be the n_9 seq
assertEquals(9, threads.get(leaderThread).seq);
// cleanup any threads still running
for (ClientThread thread : threads) {
thread.close();
thread.interrupt();
}
for (Thread thread : threads) {
thread.join();
}
}
private void waitForLeader(List<ClientThread> threads, int seq)
throws KeeperException, InterruptedException {
int leaderThread;
int tries = 0;
leaderThread = getLeaderThread();
while (threads.get(leaderThread).seq < seq) {
leaderThread = getLeaderThread();
if (tries++ > 50) {
break;
}
Thread.sleep(200);
}
}
private int getLeaderThread() throws KeeperException, InterruptedException {
String leaderUrl = getLeaderUrl("collection1", "shard1");
return Integer.parseInt(leaderUrl.replaceAll("/", ""));
}
@Test
public void testStressElection() throws Exception {
final ScheduledExecutorService scheduler = Executors
.newScheduledThreadPool(15);
final List<ClientThread> threads = Collections
.synchronizedList(new ArrayList<ClientThread>());
// start with a leader
ClientThread thread1 = null;
thread1 = new ClientThread(0);
threads.add(thread1);
scheduler.schedule(thread1, 0, TimeUnit.MILLISECONDS);
Thread.sleep(4000);
Thread scheduleThread = new Thread() {
@Override
public void run() {
for (int i = 1; i < atLeast(15); i++) {
int launchIn = random.nextInt(500);
ClientThread thread = null;
try {
thread = new ClientThread(i);
} catch (Exception e) {
//
}
if (thread != null) {
threads.add(thread);
scheduler.schedule(thread, launchIn, TimeUnit.MILLISECONDS);
}
}
}
};
Thread killThread = new Thread() {
@Override
public void run() {
while (!stopStress) {
try {
int j;
try {
// always 1 we won't kill...
j = random.nextInt(threads.size() - 2);
} catch(IllegalArgumentException e) {
continue;
}
try {
threads.get(j).close();
} catch (InterruptedException e) {
throw e;
} catch (Exception e) {
}
Thread.sleep(10);
} catch (Exception e) {
}
}
}
};
Thread connLossThread = new Thread() {
@Override
public void run() {
while (!stopStress) {
try {
Thread.sleep(50);
int j;
j = random.nextInt(threads.size());
try {
threads.get(j).zkClient.getSolrZooKeeper().pauseCnxn(
ZkTestServer.TICK_TIME * 2);
} catch (Exception e) {
e.printStackTrace();
}
Thread.sleep(500);
} catch (Exception e) {
}
}
}
};
scheduleThread.start();
connLossThread.start();
killThread.start();
Thread.sleep(6000);
stopStress = true;
scheduleThread.interrupt();
connLossThread.interrupt();
killThread.interrupt();
scheduleThread.join();
connLossThread.join();
killThread.join();
scheduler.shutdownNow();
int seq = threads.get(getLeaderThread()).getSeq();
assertFalse("seq is -1 and we may have a zombie leader", seq == -1);
// cleanup any threads still running
for (ClientThread thread : threads) {
thread.close();
}
for (Thread thread : threads) {
thread.join();
}
}
@Override
public void tearDown() throws Exception {
zkClient.close();
server.shutdown();
super.tearDown();
}
private void printLayout(String zkHost) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, AbstractZkTestCase.TIMEOUT);
zkClient.printLayoutToStdOut();
zkClient.close();
}
}

View File

@ -0,0 +1,521 @@
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.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.SolrTestCaseJ4;
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.ZkStateReader;
import org.apache.solr.core.CoreDescriptor;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
import org.junit.BeforeClass;
import org.junit.Test;
public class OverseerTest extends SolrTestCaseJ4 {
static final int TIMEOUT = 10000;
private static final boolean DEBUG = false;
@BeforeClass
public static void beforeClass() throws Exception {
initCore();
}
@Test
public void testShardAssignment() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = new ZkTestServer(zkDir);
ZkController zkController = null;
SolrZkClient zkClient = null;
try {
server.run();
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
ZkStateReader reader = new ZkStateReader(zkClient);
System.setProperty(ZkStateReader.NUM_SHARDS_PROP, "3");
zkController = new ZkController(null, server.getZkAddress(), TIMEOUT, 10000,
"localhost", "8983", "solr", new CurrentCoreDescriptorProvider() {
@Override
public List<CoreDescriptor> getCurrentDescriptors() {
// do nothing
return null;
}
});
System.setProperty("bootstrap_confdir", getFile("solr/conf")
.getAbsolutePath());
final int numShards=6;
final String[] ids = new String[numShards];
for (int i = 0; i < numShards; i++) {
CloudDescriptor collection1Desc = new CloudDescriptor();
collection1Desc.setCollectionName("collection1");
CoreDescriptor desc1 = new CoreDescriptor(null, "core" + (i + 1), "");
desc1.setCloudDescriptor(collection1Desc);
zkController.publishAsDown(zkController.getBaseUrl(), desc1,
zkController.getNodeName() + "_" + "core" + (i + 1), "core"
+ (i + 1));
ids[i] = zkController.register("core" + (i + 1), desc1);
}
assertEquals("shard1", ids[0]);
assertEquals("shard2", ids[1]);
assertEquals("shard3", ids[2]);
assertEquals("shard1", ids[3]);
assertEquals("shard2", ids[4]);
assertEquals("shard3", ids[5]);
waitForSliceCount(reader, "collection1", 3);
//make sure leaders are in cloud state
assertNotNull(reader.getLeaderUrl("collection1", "shard1", 15000));
assertNotNull(reader.getLeaderUrl("collection1", "shard2", 15000));
assertNotNull(reader.getLeaderUrl("collection1", "shard3", 15000));
} finally {
if (DEBUG) {
if (zkController != null) {
zkClient.printLayoutToStdOut();
}
}
if (zkClient != null) {
zkClient.close();
}
if (zkController != null) {
zkController.close();
}
server.shutdown();
}
System.clearProperty(ZkStateReader.NUM_SHARDS_PROP);
}
@Test
public void testShardAssignmentBigger() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
final int nodeCount = random.nextInt(50)+50; //how many simulated nodes (num of threads)
final int coreCount = random.nextInt(100)+100; //how many cores to register
final int sliceCount = random.nextInt(20)+1; //how many slices
ZkTestServer server = new ZkTestServer(zkDir);
SolrZkClient zkClient = null;
ZkStateReader reader = null;
final ZkController[] controllers = new ZkController[nodeCount];
try {
server.run();
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
reader = new ZkStateReader(zkClient);
System.setProperty(ZkStateReader.NUM_SHARDS_PROP, Integer.valueOf(sliceCount).toString());
for (int i = 0; i < nodeCount; i++) {
controllers[i] = new ZkController(null, server.getZkAddress(), TIMEOUT, 10000,
"localhost", "898" + i, "solr", new CurrentCoreDescriptorProvider() {
@Override
public List<CoreDescriptor> getCurrentDescriptors() {
// do nothing
return null;
}
});
}
System.setProperty("bootstrap_confdir", getFile("solr/conf")
.getAbsolutePath());
final ExecutorService[] nodeExecutors = new ExecutorService[nodeCount];
for (int i = 0; i < nodeCount; i++) {
nodeExecutors[i] = Executors.newFixedThreadPool(1);
}
final String[] ids = new String[coreCount];
//register total of coreCount cores
for (int i = 0; i < coreCount; i++) {
final int slot = i;
Runnable coreStarter = new Runnable() {
@Override
public void run() {
final CloudDescriptor collection1Desc = new CloudDescriptor();
collection1Desc.setCollectionName("collection1");
final String coreName = "core" + slot;
final CoreDescriptor desc = new CoreDescriptor(null, coreName, "");
desc.setCloudDescriptor(collection1Desc);
try {
controllers[slot % nodeCount].publishAsDown(controllers[slot
% nodeCount].getBaseUrl(), desc, controllers[slot
% nodeCount].getNodeName()
+ "_" + coreName, coreName);
ids[slot] = controllers[slot % nodeCount]
.register(coreName, desc);
} catch (Throwable e) {
e.printStackTrace();
fail("register threw exception:" + e.getClass());
}
}
};
nodeExecutors[i % nodeCount].submit(coreStarter);
}
for (int i = 0; i < nodeCount; i++) {
nodeExecutors[i].shutdown();
}
for (int i = 0; i < nodeCount; i++) {
while (!nodeExecutors[i].awaitTermination(100, TimeUnit.MILLISECONDS));
}
// make sure all cores have been assigned a id in cloudstate
for (int i = 0; i < 40; i++) {
reader.updateCloudState(true);
CloudState state = reader.getCloudState();
Map<String,Slice> slices = state.getSlices("collection1");
int count = 0;
for (String name : slices.keySet()) {
count += slices.get(name).getShards().size();
}
if (coreCount == count) break;
Thread.sleep(200);
}
// make sure all cores have been returned a id
for (int i = 0; i < 150; i++) {
int assignedCount = 0;
for (int j = 0; j < coreCount; j++) {
if (ids[j] != null) {
assignedCount++;
}
}
if (coreCount == assignedCount) {
break;
}
Thread.sleep(200);
}
final HashMap<String, AtomicInteger> counters = new HashMap<String,AtomicInteger>();
for (int i = 1; i < sliceCount+1; i++) {
counters.put("shard" + i, new AtomicInteger());
}
for (int i = 0; i < coreCount; i++) {
final AtomicInteger ai = counters.get(ids[i]);
assertNotNull("could not find counter for shard:" + ids[i], ai);
ai.incrementAndGet();
}
for (String counter: counters.keySet()) {
int count = counters.get(counter).intValue();
int expectedCount = coreCount / sliceCount;
int min = expectedCount - 1;
int max = expectedCount + 1;
if (count < min || count > max) {
fail("Unevenly assigned shard ids, " + counter + " had " + count
+ ", expected: " + min + "-" + max);
}
}
//make sure leaders are in cloud state
for (int i = 0; i < sliceCount; i++) {
assertNotNull(reader.getLeaderUrl("collection1", "shard" + (i + 1)), 15000);
}
} finally {
if (DEBUG) {
if (controllers[0] != null) {
zkClient.printLayoutToStdOut();
}
}
if (zkClient != null) {
zkClient.close();
}
if (reader != null) {
reader.close();
}
for (int i = 0; i < controllers.length; i++)
if (controllers[i] != null) {
controllers[i].close();
}
server.shutdown();
}
System.clearProperty(ZkStateReader.NUM_SHARDS_PROP);
}
//wait until i slices for collection have appeared
private void waitForSliceCount(ZkStateReader stateReader, String collection, int i) throws InterruptedException, KeeperException {
waitForCollections(stateReader, collection);
int maxIterations = 200;
while (0 < maxIterations--) {
CloudState state = stateReader.getCloudState();
Map<String,Slice> sliceMap = state.getSlices(collection);
if (sliceMap != null && sliceMap.keySet().size() == i) {
return;
}
Thread.sleep(100);
}
}
//wait until collections are available
private void waitForCollections(ZkStateReader stateReader, String... collections) throws InterruptedException, KeeperException {
int maxIterations = 100;
while (0 < maxIterations--) {
stateReader.updateCloudState(true);
final CloudState state = stateReader.getCloudState();
Set<String> availableCollections = state.getCollections();
int availableCount = 0;
for(String requiredCollection: collections) {
if(availableCollections.contains(requiredCollection)) {
availableCount++;
}
if(availableCount == collections.length) return;
Thread.sleep(50);
}
}
log.warn("Timeout waiting for collections: " + Arrays.asList(collections) + " state:" + stateReader.getCloudState());
}
@Test
public void testStateChange() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = new ZkTestServer(zkDir);
SolrZkClient zkClient = null;
ZkStateReader reader = null;
SolrZkClient overseerClient = null;
try {
server.run();
zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient.makePath("/live_nodes", true);
System.setProperty(ZkStateReader.NUM_SHARDS_PROP, "2");
//live node
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + "node1";
zkClient.makePath(nodePath,CreateMode.EPHEMERAL, true);
reader = new ZkStateReader(zkClient);
reader.createClusterStateWatchersAndUpdate();
Overseer.createClientNodes(zkClient, "node1");
overseerClient = electNewOverseer(server.getZkAddress());
HashMap<String, String> coreProps = new HashMap<String,String>();
coreProps.put(ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr");
coreProps.put(ZkStateReader.NODE_NAME_PROP, "node1");
coreProps.put(ZkStateReader.CORE_NAME_PROP, "core1");
coreProps.put(ZkStateReader.ROLES_PROP, "");
coreProps.put(ZkStateReader.STATE_PROP, ZkStateReader.RECOVERING);
CoreState state = new CoreState("core1", "collection1", coreProps);
nodePath = "/node_states/node1";
try {
zkClient.makePath(nodePath, CreateMode.EPHEMERAL, true);
} catch (KeeperException ke) {
if(ke.code()!=Code.NODEEXISTS) {
throw ke;
}
}
//publish node state (recovering)
zkClient.setData(nodePath, ZkStateReader.toJSON(new CoreState[]{state}), true);
//wait overseer assignment
waitForSliceCount(reader, "collection1", 1);
assertEquals(reader.getCloudState().toString(), ZkStateReader.RECOVERING,
reader.getCloudState().getSlice("collection1", "shard1").getShards()
.get("node1_core1").get(ZkStateReader.STATE_PROP));
//publish node state (active)
coreProps.put(ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE);
coreProps.put(ZkStateReader.SHARD_ID_PROP, "shard1");
state = new CoreState("core1", "collection1", coreProps);
zkClient.setData(nodePath, ZkStateReader.toJSON(new CoreState[]{state}), true);
verifyStatus(reader, ZkStateReader.ACTIVE);
} finally {
System.clearProperty(ZkStateReader.NUM_SHARDS_PROP);
if (zkClient != null) {
zkClient.close();
}
if (overseerClient != null) {
overseerClient.close();
}
if (reader != null) {
reader.close();
}
server.shutdown();
}
}
private void verifyStatus(ZkStateReader reader, String expectedState) throws InterruptedException {
int maxIterations = 100;
String coreState = null;
while(maxIterations-->0) {
coreState = reader.getCloudState().getSlice("collection1", "shard1").getShards().get("node1_core1").get(ZkStateReader.STATE_PROP);
if(coreState.equals(expectedState)) {
return;
}
Thread.sleep(50);
}
fail("Illegal state, was:" + coreState + " expected:" + expectedState + "cloudState:" + reader.getCloudState());
}
@Test
public void testOverseerFailure() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = new ZkTestServer(zkDir);
SolrZkClient controllerClient = null;
SolrZkClient overseerClient = null;
ZkStateReader reader = null;
try {
server.run();
controllerClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
controllerClient.makePath(ZkStateReader.LIVE_NODES_ZKNODE, true);
reader = new ZkStateReader(controllerClient);
reader.createClusterStateWatchersAndUpdate();
Overseer.createClientNodes(controllerClient, "node1");
overseerClient = electNewOverseer(server.getZkAddress());
// live node
final String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + "node1";
controllerClient.makePath(nodePath, CreateMode.EPHEMERAL, true);
HashMap<String,String> coreProps = new HashMap<String,String>();
coreProps.put(ZkStateReader.STATE_PROP, ZkStateReader.RECOVERING);
coreProps.put(ZkStateReader.NODE_NAME_PROP, "node1");
CoreState state = new CoreState("core1", "collection1", coreProps);
final String statePath = Overseer.STATES_NODE + "/node1";
// publish node state (recovering)
controllerClient.setData(statePath, ZkStateReader.toJSON(new CoreState[] {state}), true);
// wait overseer assignment
waitForSliceCount(reader, "collection1", 1);
verifyStatus(reader, ZkStateReader.RECOVERING);
// publish node state (active)
coreProps.put(ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE);
coreProps.put(ZkStateReader.SHARD_ID_PROP, "shard1");
state = new CoreState("core1", "collection1", coreProps);
controllerClient.setData(statePath,
ZkStateReader.toJSON(new CoreState[] {state}), true);
verifyStatus(reader, ZkStateReader.ACTIVE);
overseerClient.close();
coreProps.put(ZkStateReader.STATE_PROP, ZkStateReader.RECOVERING);
state = new CoreState("core1", "collection1", coreProps);
controllerClient.setData(statePath,
ZkStateReader.toJSON(new CoreState[] {state}), true);
overseerClient = electNewOverseer(server.getZkAddress());
verifyStatus(reader, ZkStateReader.RECOVERING);
assertEquals("Live nodes count does not match", 1, reader.getCloudState()
.getLiveNodes().size());
assertEquals("Shard count does not match", 1, reader.getCloudState()
.getSlice("collection1", "shard1").getShards().size());
} finally {
if (overseerClient != null) {
overseerClient.close();
}
if (controllerClient != null) {
controllerClient.close();
}
if (reader != null) {
reader.close();
}
server.shutdown();
}
}
private SolrZkClient electNewOverseer(String address) throws InterruptedException,
TimeoutException, IOException, KeeperException {
SolrZkClient zkClient = new SolrZkClient(address, TIMEOUT);
ZkStateReader reader = new ZkStateReader(zkClient);
LeaderElector overseerElector = new LeaderElector(zkClient);
ElectionContext ec = new OverseerElectionContext(address.replaceAll("/", "_"), zkClient, reader);
overseerElector.setup(ec);
overseerElector.joinElection(ec);
return zkClient;
}
}

View File

@ -0,0 +1,142 @@
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 org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.common.SolrInputDocument;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class RecoveryZkTest extends FullSolrCloudTest {
//private static final String DISTRIB_UPDATE_CHAIN = "distrib-update-chain";
private static Logger log = LoggerFactory.getLogger(RecoveryZkTest.class);
private StopableIndexingThread indexThread;
private StopableIndexingThread indexThread2;
@BeforeClass
public static void beforeSuperClass() throws Exception {
}
@AfterClass
public static void afterSuperClass() throws Exception {
}
public RecoveryZkTest() {
super();
sliceCount = 1;
shardCount = 2;
}
@Override
public void doTest() throws Exception {
handle.clear();
handle.put("QTime", SKIPVAL);
handle.put("timestamp", SKIPVAL);
// start a couple indexing threads
indexThread = new StopableIndexingThread(0, true);
indexThread.start();
indexThread2 = new StopableIndexingThread(10000, true);
indexThread2.start();
// give some time to index...
Thread.sleep(atLeast(2000));
// bring shard replica down
JettySolrRunner replica = chaosMonkey.stopShard("shard1", 1);
// wait a moment - lets allow some docs to be indexed so replication time is non 0
Thread.sleep(atLeast(2000));
// bring shard replica up
replica.start();
// make sure replication can start
Thread.sleep(1500);
waitForRecoveriesToFinish(false);
// stop indexing threads
indexThread.safeStop();
indexThread2.safeStop();
indexThread.join();
indexThread2.join();
Thread.sleep(5000);
commit();
// test that leader and replica have same doc count
checkShardConsistency("shard1", false);
long client1Docs = shardToClient.get("shard1").get(0).query(new SolrQuery("*:*")).getResults().getNumFound();
long client2Docs = shardToClient.get("shard1").get(1).query(new SolrQuery("*:*")).getResults().getNumFound();
assertTrue(client1Docs > 0);
assertEquals(client1Docs, client2Docs);
// won't always pass yet...
//query("q", "*:*", "sort", "id desc");
}
protected void indexDoc(SolrInputDocument doc) throws IOException,
SolrServerException {
controlClient.add(doc);
// UpdateRequest ureq = new UpdateRequest();
// ureq.add(doc);
// ureq.setParam("update.chain", DISTRIB_UPDATE_CHAIN);
// ureq.process(cloudClient);
cloudClient.add(doc);
}
@Override
public void tearDown() throws Exception {
// make sure threads have been stopped...
indexThread.safeStop();
indexThread2.safeStop();
indexThread.join();
indexThread2.join();
super.tearDown();
}
// skip the randoms - they can deadlock...
protected void indexr(Object... fields) throws Exception {
SolrInputDocument doc = new SolrInputDocument();
addFields(doc, fields);
addFields(doc, "rnd_b", true);
indexDoc(doc);
}
}

View File

@ -0,0 +1,45 @@
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.SolrTestCaseJ4;
import org.apache.solr.common.cloud.HashPartitioner;
import org.apache.solr.common.cloud.HashPartitioner.Range;
public class TestHashPartitioner extends SolrTestCaseJ4 {
public void testMapHashes() throws Exception {
HashPartitioner hp = new HashPartitioner();
for (int i = 1; i <= 30000; i++) {
List<Range> ranges = hp.partitionRange(i);
assertEquals(i, ranges.size());
assertTrue("First range does not start before " + Integer.MIN_VALUE
+ " it is:" + ranges.get(0).min,
ranges.get(0).min <= Integer.MIN_VALUE);
assertTrue("Last range does not end after " + Integer.MAX_VALUE
+ " it is:" + ranges.get(ranges.size() - 1).max,
ranges.get(ranges.size() - 1).max >= Integer.MAX_VALUE);
}
}
}

View File

@ -18,41 +18,26 @@ package org.apache.solr.cloud;
*/
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.solr.SolrTestCaseJ4;
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.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrConfig;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
public class ZkControllerTest extends SolrTestCaseJ4 {
private static final String TEST_NODE_NAME = "test_node_name";
private static final String URL3 = "http://localhost:3133/solr/core1";
private static final String URL2 = "http://localhost:3123/solr/core1";
private static final String SHARD3 = "localhost:3123_solr_core3";
private static final String SHARD2 = "localhost:3123_solr_core2";
private static final String SHARD1 = "localhost:3123_solr_core1";
private static final String COLLECTION_NAME = "collection1";
static final int TIMEOUT = 10000;
private static final String URL1 = "http://localhost:3133/solr/core0";
static final int TIMEOUT = 1000;
private static final boolean DEBUG = false;
@ -61,79 +46,6 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
initCore();
}
@Test
public void testReadShards() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = null;
SolrZkClient zkClient = null;
ZkController zkController = null;
try {
server = new ZkTestServer(zkDir);
server.run();
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
String shardsPath = "/collections/collection1/shards/shardid1";
zkClient.makePath(shardsPath);
addShardToZk(zkClient, shardsPath, SHARD1, URL1);
addShardToZk(zkClient, shardsPath, SHARD2, URL2);
addShardToZk(zkClient, shardsPath, SHARD3, URL3);
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
zkController = new ZkController(server.getZkAddress(),
TIMEOUT, 1000, "localhost", "8983", "solr");
zkController.getZkStateReader().updateCloudState(true);
CloudState cloudInfo = zkController.getCloudState();
Map<String,Slice> slices = cloudInfo.getSlices("collection1");
assertNotNull(slices);
for (Slice slice : slices.values()) {
Map<String,ZkNodeProps> shards = slice.getShards();
if (DEBUG) {
for (String shardName : shards.keySet()) {
ZkNodeProps props = shards.get(shardName);
System.out.println("shard:" + shardName);
System.out.println("props:" + props.toString());
}
}
assertNotNull(shards.get(SHARD1));
assertNotNull(shards.get(SHARD2));
assertNotNull(shards.get(SHARD3));
ZkNodeProps props = shards.get(SHARD1);
assertEquals(URL1, props.get(ZkStateReader.URL_PROP));
assertEquals(TEST_NODE_NAME, props.get(ZkStateReader.NODE_NAME));
props = shards.get(SHARD2);
assertEquals(URL2, props.get(ZkStateReader.URL_PROP));
assertEquals(TEST_NODE_NAME, props.get(ZkStateReader.NODE_NAME));
props = shards.get(SHARD3);
assertEquals(URL3, props.get(ZkStateReader.URL_PROP));
assertEquals(TEST_NODE_NAME, props.get(ZkStateReader.NODE_NAME));
}
} finally {
if (zkClient != null) {
zkClient.close();
}
if (zkController != null) {
zkController.close();
}
if (server != null) {
server.shutdown();
}
}
}
@Test
public void testReadConfigName() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
@ -148,18 +60,28 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
String actualConfigName = "firstConfig";
zkClient.makePath(ZkController.CONFIGS_ZKNODE + "/" + actualConfigName);
zkClient.makePath(ZkController.CONFIGS_ZKNODE + "/" + actualConfigName, true);
ZkNodeProps props = new ZkNodeProps();
Map<String,String> props = new HashMap<String,String>();
props.put("configName", actualConfigName);
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + COLLECTION_NAME , props.store(), CreateMode.PERSISTENT);
ZkNodeProps zkProps = new ZkNodeProps(props);
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/"
+ COLLECTION_NAME, ZkStateReader.toJSON(zkProps),
CreateMode.PERSISTENT, true);
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
zkClient.close();
ZkController zkController = new ZkController(server.getZkAddress(), TIMEOUT, TIMEOUT,
"localhost", "8983", "/solr");
ZkController zkController = new ZkController(null, server.getZkAddress(), TIMEOUT, 10000,
"localhost", "8983", "solr", new CurrentCoreDescriptorProvider() {
@Override
public List<CoreDescriptor> getCurrentDescriptors() {
// do nothing
return null;
}
});
try {
String configName = zkController.readConfigName(COLLECTION_NAME);
assertEquals(configName, actualConfigName);
@ -185,9 +107,20 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkController = new ZkController(server.getZkAddress(),
TIMEOUT, 10000, "localhost", "8983", "/solr");
zkController = new ZkController(null, server.getZkAddress(),
TIMEOUT, 10000, "localhost", "8983", "solr", new CurrentCoreDescriptorProvider() {
@Override
public List<CoreDescriptor> getCurrentDescriptors() {
// do nothing
return null;
}
});
zkController.uploadToZK(getFile("solr/conf"),
ZkController.CONFIGS_ZKNODE + "/config1");
// uploading again should overwrite, not error...
zkController.uploadToZK(getFile("solr/conf"),
ZkController.CONFIGS_ZKNODE + "/config1");
@ -204,21 +137,14 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
}
private void addShardToZk(SolrZkClient zkClient, String shardsPath,
String zkNodeName, String url) throws IOException,
KeeperException, InterruptedException {
ZkNodeProps props = new ZkNodeProps();
props.put(ZkStateReader.URL_PROP, url);
props.put(ZkStateReader.NODE_NAME, TEST_NODE_NAME);
byte[] bytes = props.store();
zkClient
.create(shardsPath + "/" + zkNodeName, bytes, CreateMode.PERSISTENT);
}
@Override
public void tearDown() throws Exception {
super.tearDown();
}
@AfterClass
public static void afterClass() throws InterruptedException {
// wait just a bit for any zk client threads to outlast timeout
Thread.sleep(2000);
}
}

View File

@ -18,27 +18,30 @@ package org.apache.solr.cloud;
*/
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.junit.Test;
public class ZkNodePropsTest extends SolrTestCaseJ4 {
@Test
public void testBasic() throws IOException {
ZkNodeProps props = new ZkNodeProps();
Map<String,String> props = new HashMap<String,String>();
props.put("prop1", "value1");
props.put("prop2", "value2");
props.put("prop3", "value3");
props.put("prop4", "value4");
props.put("prop5", "value5");
props.put("prop6", "value6");
byte[] bytes = props.store();
ZkNodeProps props2 = new ZkNodeProps();
props2.load(bytes);
ZkNodeProps zkProps = new ZkNodeProps(props);
byte[] bytes = ZkStateReader.toJSON(zkProps);
ZkNodeProps props2 = ZkNodeProps.load(bytes);
assertEquals("value1", props2.get("prop1"));
assertEquals("value2", props2.get("prop2"));
assertEquals("value3", props2.get("prop3"));

View File

@ -28,6 +28,7 @@ import org.apache.solr.util.AbstractSolrTestCase;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.junit.AfterClass;
public class ZkSolrClientTest extends AbstractSolrTestCase {
private static final boolean DEBUG = false;
@ -59,7 +60,7 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
SolrZkClient zkClient = new SolrZkClient(server.getZkHost(),
AbstractZkTestCase.TIMEOUT);
assertTrue(zkClient.exists("/solr"));
assertTrue(zkClient.exists("/solr", true));
zkClient.close();
server.shutdown();
@ -78,9 +79,9 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
zkClient = new SolrZkClient(server.getZkAddress(), AbstractZkTestCase.TIMEOUT);
String shardsPath = "/collections/collection1/shards";
zkClient.makePath(shardsPath);
zkClient.makePath(shardsPath, false, true);
zkClient.makePath("collections/collection1");
zkClient.makePath("collections/collection1", false, true);
int zkServerPort = server.getPort();
// this tests disconnect state
server.shutdown();
@ -89,7 +90,7 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
try {
zkClient.makePath("collections/collection2");
zkClient.makePath("collections/collection2", false);
TestCase.fail("Server should be down here");
} catch (KeeperException.ConnectionLossException e) {
@ -104,18 +105,18 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
Thread.sleep(600);
try {
zkClient.makePath("collections/collection3");
zkClient.makePath("collections/collection3", true);
} catch (KeeperException.ConnectionLossException e) {
Thread.sleep(5000); // try again in a bit
zkClient.makePath("collections/collection3");
zkClient.makePath("collections/collection3", true);
}
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
assertNotNull(zkClient.exists("/collections/collection3", null));
assertNotNull(zkClient.exists("/collections/collection1", null));
assertNotNull(zkClient.exists("/collections/collection3", null, true));
assertNotNull(zkClient.exists("/collections/collection1", null, true));
// simulate session expiration
@ -132,7 +133,7 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
for (int i = 0; i < 8; i++) {
try {
zkClient.makePath("collections/collection4");
zkClient.makePath("collections/collection4", true);
break;
} catch (KeeperException.SessionExpiredException e) {
@ -146,7 +147,7 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
zkClient.printLayoutToStdOut();
}
assertNotNull("Node does not exist, but it should", zkClient.exists("/collections/collection4", null));
assertNotNull("Node does not exist, but it should", zkClient.exists("/collections/collection4", null, true));
} finally {
@ -170,7 +171,7 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
final SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), AbstractZkTestCase.TIMEOUT);
try {
zkClient.makePath("/collections");
zkClient.makePath("/collections", true);
zkClient.getChildren("/collections", new Watcher() {
@ -181,22 +182,22 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
cnt.incrementAndGet();
// remake watch
try {
zkClient.getChildren("/collections", this);
zkClient.getChildren("/collections", this, true);
} catch (KeeperException e) {
throw new RuntimeException(e);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
});
}, true);
zkClient.makePath("/collections/collection99/shards");
zkClient.makePath("/collections/collection99/shards", true);
zkClient.makePath("collections/collection99/config=collection1");
zkClient.makePath("collections/collection99/config=collection1", true);
zkClient.makePath("collections/collection99/config=collection3");
zkClient.makePath("collections/collection99/config=collection3", true);
zkClient.makePath("/collections/collection97/shards");
zkClient.makePath("/collections/collection97/shards", true);
if (DEBUG) {
zkClient.printLayoutToStdOut();
@ -209,6 +210,10 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
Thread.sleep(4000); // wait a bit more
}
if (cnt.intValue() < 2) {
Thread.sleep(4000); // wait a bit more
}
assertEquals(2, cnt.intValue());
} finally {
@ -237,4 +242,9 @@ public class ZkSolrClientTest extends AbstractSolrTestCase {
super.tearDown();
}
@AfterClass
public static void afterClass() throws InterruptedException {
// wait just a bit for any zk client threads to outlast timeout
Thread.sleep(2000);
}
}

Some files were not shown because too many files have changed in this diff Show More