You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-user@lucene.apache.org by "A.Eibner" <a_...@yahoo.de> on 2013/04/08 15:34:35 UTC

Empty Solr 4.2.1 can not create Collection

Hi,

I have a problem with setting up my solr cloud environment (on three 
machines).
If I want to create my collections from scratch I do the following:

*) Start ZooKeeper on all machines.

*) Upload the configuration (on app02) for the collection via the 
following command:
     zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:4181 
--confdir config/solr/storage/conf/ --confname storage-conf

*) Linking the configuration (on app02) via the following command:
     zkcli.sh -cmd linkconfig --collection storage --confname 
storage-conf --zkhost app01:4181,app02:4181,app03:4181

*) Start Tomcats (containing Solr) on app02,app03

*) Create Collection via:
http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf

This creates the replication of the shard on app02 and app03, but 
neither of them is marked as leader, both are marked as DOWN.
And after wards I can not access the collection.
In the browser I get:
"SEVERE: org.apache.solr.common.SolrException: no servers hosting shard:"

In the log files the following error is present:
SEVERE: Error from shard: app02:9985/solr
org.apache.solr.common.SolrException: Error CREATEing SolrCore 
'storage_shard1_replica1':
         at 
org.apache.solr.client.solrj.impl.HttpSolrServer.request(HttpSolrServer.java:404)
         at 
org.apache.solr.client.solrj.impl.HttpSolrServer.request(HttpSolrServer.java:181)
         at 
org.apache.solr.handler.component.HttpShardHandler$1.call(HttpShardHandler.java:172)
         at 
org.apache.solr.handler.component.HttpShardHandler$1.call(HttpShardHandler.java:135)
         at 
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
         at java.util.concurrent.FutureTask.run(FutureTask.java:166)
         at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
         at 
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
         at java.util.concurrent.FutureTask.run(FutureTask.java:166)
         at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
         at java.lang.Thread.run(Thread.java:722)
Caused by: org.apache.solr.common.cloud.ZooKeeperException:
         at 
org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:922)
         at 
org.apache.solr.core.CoreContainer.registerCore(CoreContainer.java:892)
         at 
org.apache.solr.core.CoreContainer.register(CoreContainer.java:841)
         at 
org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:479)
         ... 19 more
Caused by: org.apache.solr.common.SolrException: Error getting leader 
from zk for shard shard1
         at 
org.apache.solr.cloud.ZkController.getLeader(ZkController.java:864)
         at 
org.apache.solr.cloud.ZkController.register(ZkController.java:776)
         at 
org.apache.solr.cloud.ZkController.register(ZkController.java:727)
         at 
org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:908)
         ... 22 more
Caused by: java.lang.InterruptedException: sleep interrupted
         at java.lang.Thread.sleep(Native Method)
         at 
org.apache.solr.cloud.ZkController.getLeaderProps(ZkController.java:905)
         at 
org.apache.solr.cloud.ZkController.getLeaderProps(ZkController.java:875)
         at 
org.apache.solr.cloud.ZkController.getLeader(ZkController.java:839)
         ... 25 more

I have attached a minimal set of configuration files which are needed to 
replicate this error, also containing the log files for the commands I 
have run in the order above.

I use the following versions of:
Solr: 4.2.1
ZooKeeper: 3.4.5
Tomcat 7.0.27

app01: ZooKeeper
app02: ZooKeeper, Solr (in Tomcat)
app03: ZooKeeper, Solr (in Tomcat)

The same procedure does work if I use Solr 4.0, but the BUG 
(https://issues.apache.org/jira/browse/SOLR-3939
) prevents me from using 4.0 in this scenario.

If you need anything else, please just say so.

Thanks for your help

Kind Regards
Alexander


Re: Using the Collections API

Posted by Shawn Heisey <so...@elyograg.org>.
On 5/15/2013 4:53 AM, A.Eibner wrote:
> I just wanted to ask, if anyone is using the collections API to create
> collections,
> or if not how they use the coreAPI to create a collection with
> replication ?

For my little SolrCloud install using 4.2.1, I have used the collections
API exclusively.  It has worked perfectly.

I updated SOLR-4734.

Thanks,
Shawn


Re: Using the Collections API

Posted by Mark Miller <ma...@gmail.com>.
What version of Solr? I think there was a bug a couple versions back (perhaps introduced in 4.1 if I remember right) that made it so creates were not spread correctly.

- Mark

Re: Using the Collections API

Posted by Shawn Heisey <so...@elyograg.org>.
On 5/17/2013 4:03 AM, Jared Rodriguez wrote:
> So it sounds like you want the collection created with a master and a
> replica and you want one to be on each node?  If so, I believe that you can
> get that effect by specifying maxShardsPerNode=1 as part of your url line.
>  This will tell solr to create the master and replica that you desire but
> to not put them on the same node.

Jared,

The default value of maxShardsPerNode is 1.  Even if you make it higher,
the collections API won't put replicas for the same shard on the same
server, as long as the number of available hosts is a multiple of your
replication factor.

I would consider it a bug if the API were to ever put more than one
replica for the same shard on the same host, regardless of the number of
available hosts.  I do not know whether the API has such a bug or not.

A override option to allow replicas on the same host might make sense
for proof-of-concept setups, but you'd never want to use it in production.

Thanks,
Shawn


Re: Using the Collections API

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi Jared,

yes that is what I want to achieve:
Creating a master and a replica and I want them to be separate nodes.

I just realized that I posted the wrong URL,  I was already using the 
parameter maxShardsPerNode=1.

But just to be sure, I also tried it with your URL and I get the same 
result.
Both replicas will be created on app02.
It does not matter where I create the collection (meaning app02 or 
app03) via the collections api.

And as I said, both nodes appear live in zookeeper

[zk: localhost:4181(CONNECTED) 1] ls /live_nodes
[app02:9985_solr, app03:9985_solr]

Any clue?

Regards
Alexander


Am 2013-05-17 12:03, schrieb Jared Rodriguez:
> Hi Alexander,
>
> So it sounds like you want the collection created with a master and a
> replica and you want one to be on each node?  If so, I believe that you can
> get that effect by specifying maxShardsPerNode=1 as part of your url line.
>   This will tell solr to create the master and replica that you desire but
> to not put them on the same node.
>
> Your url would look like:
>
> http://app02:9985/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf&maxShardsPerNode=1
>
> The SolrCloud wiki does a good job of explaining the params and how they
> function.
> http://wiki.apache.org/solr/SolrCloud
>
>
> Jared
>
>
>
> On Fri, May 17, 2013 at 4:57 AM, A.Eibner <a_...@yahoo.de> wrote:
>
>> Hi, sorry for the delay.
>>
>> I have two live nodes (also zookeeper knows these two
>> [app02:9985_solrl,app03:9985_**solr])
>>
>> But when I want to create a collection via:
>>
>> http://app02:9985/solr/admin/**collections?action=CREATE&**
>> name=storage&numShards=1&**replicationFactor=2&**
>> collection.configName=storage-**conf<http://app02:9985/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>
>> Both replicas will be created on app02.
>>
>> Any clues ?
>> Should I post anything else?
>>
>> Regards
>> Alexander
>>
>> Am 2013-05-15 14:48, schrieb Mark Miller:
>>
>>   Yeah, I use both on an empty Solr - what is the error?
>>>
>>> - Mark
>>>
>>> On May 15, 2013, at 6:53 AM, A.Eibner <a_...@yahoo.de> wrote:
>>>
>>>   Hi,
>>>>
>>>> I just wanted to ask, if anyone is using the collections API to create
>>>> collections,
>>>> or if not how they use the coreAPI to create a collection with
>>>> replication ?
>>>>
>>>> Because I run into errors when creating a collection on an empty solr.
>>>>
>>>> Kind regards
>>>> Alexander
>>>>
>>>
>>>
>>>
>>
>
>


Re: Using the Collections API

Posted by Jared Rodriguez <jr...@kitedesk.com>.
Hi Alexander,

So it sounds like you want the collection created with a master and a
replica and you want one to be on each node?  If so, I believe that you can
get that effect by specifying maxShardsPerNode=1 as part of your url line.
 This will tell solr to create the master and replica that you desire but
to not put them on the same node.

Your url would look like:

http://app02:9985/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf&maxShardsPerNode=1

The SolrCloud wiki does a good job of explaining the params and how they
function.
http://wiki.apache.org/solr/SolrCloud


Jared



On Fri, May 17, 2013 at 4:57 AM, A.Eibner <a_...@yahoo.de> wrote:

> Hi, sorry for the delay.
>
> I have two live nodes (also zookeeper knows these two
> [app02:9985_solrl,app03:9985_**solr])
>
> But when I want to create a collection via:
>
> http://app02:9985/solr/admin/**collections?action=CREATE&**
> name=storage&numShards=1&**replicationFactor=2&**
> collection.configName=storage-**conf<http://app02:9985/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>
> Both replicas will be created on app02.
>
> Any clues ?
> Should I post anything else?
>
> Regards
> Alexander
>
> Am 2013-05-15 14:48, schrieb Mark Miller:
>
>  Yeah, I use both on an empty Solr - what is the error?
>>
>> - Mark
>>
>> On May 15, 2013, at 6:53 AM, A.Eibner <a_...@yahoo.de> wrote:
>>
>>  Hi,
>>>
>>> I just wanted to ask, if anyone is using the collections API to create
>>> collections,
>>> or if not how they use the coreAPI to create a collection with
>>> replication ?
>>>
>>> Because I run into errors when creating a collection on an empty solr.
>>>
>>> Kind regards
>>> Alexander
>>>
>>
>>
>>
>


-- 
Jared Rodriguez

Re: Using the Collections API

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi, sorry for the delay.

I have two live nodes (also zookeeper knows these two 
[app02:9985_solrl,app03:9985_solr])

But when I want to create a collection via:

http://app02:9985/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf

Both replicas will be created on app02.

Any clues ?
Should I post anything else?

Regards
Alexander

Am 2013-05-15 14:48, schrieb Mark Miller:
> Yeah, I use both on an empty Solr - what is the error?
>
> - Mark
>
> On May 15, 2013, at 6:53 AM, A.Eibner <a_...@yahoo.de> wrote:
>
>> Hi,
>>
>> I just wanted to ask, if anyone is using the collections API to create collections,
>> or if not how they use the coreAPI to create a collection with replication ?
>>
>> Because I run into errors when creating a collection on an empty solr.
>>
>> Kind regards
>> Alexander
>
>


Re: Using the Collections API

Posted by Mark Miller <ma...@gmail.com>.
Yeah, I use both on an empty Solr - what is the error?

- Mark

On May 15, 2013, at 6:53 AM, A.Eibner <a_...@yahoo.de> wrote:

> Hi,
> 
> I just wanted to ask, if anyone is using the collections API to create collections,
> or if not how they use the coreAPI to create a collection with replication ?
> 
> Because I run into errors when creating a collection on an empty solr.
> 
> Kind regards
> Alexander


Re: Using the Collections API

Posted by Jared Rodriguez <jr...@kitedesk.com>.
Hi Mark,

Yes, I am using reload.  Here is the jira that I filed.

https://issues.apache.org/jira/browse/SOLR-4805

Please let me know if there is any additional data that you need.


On Wed, May 15, 2013 at 12:53 PM, Mark Miller <ma...@gmail.com> wrote:

>
> On May 15, 2013, at 12:26 PM, Jared Rodriguez <jr...@kitedesk.com>
> wrote:
>
> > the cores in the collection stay offline even if there are no
> > material changes.
>
> I've used reload - if you are having trouble with it, please post more
> details or file a JIRA issue.
>
> - Mark




-- 
Jared Rodriguez

Re: Using the Collections API

Posted by Mark Miller <ma...@gmail.com>.
On May 15, 2013, at 12:26 PM, Jared Rodriguez <jr...@kitedesk.com> wrote:

> the cores in the collection stay offline even if there are no
> material changes.

I've used reload - if you are having trouble with it, please post more details or file a JIRA issue.

- Mark

Re: Using the Collections API

Posted by Jared Rodriguez <jr...@kitedesk.com>.
I have used both and they seem to work well for basic operations - create,
delete, etc.  Although newer operations like reload do not function as they
should - the cores in the collection stay offline even if there are no
material changes.


On Wed, May 15, 2013 at 6:53 AM, A.Eibner <a_...@yahoo.de> wrote:

> Hi,
>
> I just wanted to ask, if anyone is using the collections API to create
> collections,
> or if not how they use the coreAPI to create a collection with replication
> ?
>
> Because I run into errors when creating a collection on an empty solr.
>
> Kind regards
> Alexander
>



-- 
Jared Rodriguez

Using the Collections API

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi,

I just wanted to ask, if anyone is using the collections API to create 
collections,
or if not how they use the coreAPI to create a collection with 
replication ?

Because I run into errors when creating a collection on an empty solr.

Kind regards
Alexander

Re: Empty Solr 4.2.1 can not create Collection

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi,
I filed an issue at https://issues.apache.org/jira/browse/SOLR-4734
I also tried this with 4.3, but the same error occurs.

Should I post on the dev list ?

Kind regards
Alexander

Am 2013-04-16 23:47, schrieb Chris Hostetter:
>
> : sorry for pushing, but I just replayed the steps with solr 4.0 where
> : everything works fine.
> : Then I switched to solr 4.2.1 and replayed the exact same steps and the
> : collection won't start and no leader will be elected.
> :
> : Any clues ?
> : Should I try it on the developer mailing list, maybe it's a bug ?
>
> I'm not really understanding what the sequence of events is that's leading
> you to this error, but if you can reproduce a problem in which there is no
> leader election (and you get the NPE listed below) when creating a
> collection then yes, absolutely, please open a Jira and include...
>
> 1) the specific list of steps to reproduce starting from a 4.2.1 install
> 2) the configs you start with as well as any configs you are specifying
> when creating collections
> 3) snapshots of clusterstate.json taken before and after you encounter the
> problem
> 4) logs from each of hte solr servers you run in your test.
>
>
>
> :
> : Kind Regards
> : Alexander
> :
> : Am 2013-04-10 22:27, schrieb A.Eibner:
> : > Hi,
> : >
> : > here the clusterstate.json (from zookeeper) after creating the core:
> : >
> : > {"storage":{
> : >      "shards":{"shard1":{
> : >          "range":"80000000-7fffffff",
> : >          "state":"active",
> : >          "replicas":{"app02:9985_solr_storage-core":{
> : >              "shard":"shard1",
> : >              "state":"down",
> : >              "core":"storage-core",
> : >              "collection":"storage",
> : >              "node_name":"app02:9985_solr",
> : >              "base_url":"http://app02:9985/solr"}}}},
> : >      "router":"compositeId"}}
> : > cZxid = 0x100000024
> : > ctime = Wed Apr 10 22:18:13 CEST 2013
> : > mZxid = 0x10000003d
> : > mtime = Wed Apr 10 22:21:26 CEST 2013
> : > pZxid = 0x100000024
> : > cversion = 0
> : > dataVersion = 2
> : > aclVersion = 0
> : > ephemeralOwner = 0x0
> : > dataLength = 467
> : > numChildren = 0
> : >
> : > But looking in the log files I found the following error (this also
> : > occures with the collection api)
> : >
> : > SEVERE: org.apache.solr.common.SolrException: Error CREATEing SolrCore
> : > 'storage_shard1_replica1':
> : >          at
> : > org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:483)
> : >
> : >          at
> : > org.apache.solr.handler.admin.CoreAdminHandler.handleRequestBody(CoreAdminHandler.java:140)
> : >
> : >          at
> : > org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:135)
> : >
> : >          at
> : > org.apache.solr.servlet.SolrDispatchFilter.handleAdminRequest(SolrDispatchFilter.java:591)
> : >
> : >          at
> : > org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:192)
> : >
> : >          at
> : > org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:141)
> : >
> : >          at
> : > org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:243)
> : >
> : >          at
> : > org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:210)
> : >
> : >          at
> : > org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:225)
> : >
> : >          at
> : > org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:169)
> : >
> : >          at
> : > org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:168)
> : >
> : >          at
> : > org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:98)
> : >
> : >          at
> : > org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:118)
> : >
> : >          at
> : > org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:407)
> : >          at
> : > org.apache.coyote.http11.AbstractHttp11Processor.process(AbstractHttp11Processor.java:999)
> : >
> : >          at
> : > org.apache.coyote.AbstractProtocol$AbstractConnectionHandler.process(AbstractProtocol.java:565)
> : >
> : >          at
> : > org.apache.tomcat.util.net.JIoEndpoint$SocketProcessor.run(JIoEndpoint.java:307)
> : >
> : >          at
> : > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> : >
> : >          at
> : > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> : >
> : >          at java.lang.Thread.run(Thread.java:722)
> : > Caused by: org.apache.solr.common.cloud.ZooKeeperException:
> : >          at
> : > org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:931)
> : >          at
> : > org.apache.solr.core.CoreContainer.registerCore(CoreContainer.java:892)
> : >          at
> : > org.apache.solr.core.CoreContainer.register(CoreContainer.java:841)
> : >          at
> : > org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:479)
> : >
> : >          ... 19 more
> : > Caused by: java.lang.NullPointerException
> : >          at
> : > org.apache.solr.cloud.ShardLeaderElectionContext.runLeaderProcess(ElectionContext.java:190)
> : >
> : >          at
> : > org.apache.solr.cloud.LeaderElector.runIamLeaderProcess(LeaderElector.java:156)
> : >
> : >          at
> : > org.apache.solr.cloud.LeaderElector.checkIfIamLeader(LeaderElector.java:100)
> : >
> : >          at
> : > org.apache.solr.cloud.LeaderElector.joinElection(LeaderElector.java:266)
> : >          at
> : > org.apache.solr.cloud.ZkController.joinElection(ZkController.java:935)
> : >          at
> : > org.apache.solr.cloud.ZkController.register(ZkController.java:761)
> : >          at
> : > org.apache.solr.cloud.ZkController.register(ZkController.java:727)
> : >          at
> : > org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:908)
> : >          ... 22 more
> : >
> : > Kind regards
> : > Alexander
> : >
> : > Am 2013-04-10 19:12, schrieb Joel Bernstein:
> : > > Can you post what your clusterstate.json?
> : > >
> : > > After you spin up the initial core, it will automatically become
> : > > leader for
> : > > that shard.
> : > >
> : > >
> : > > On Wed, Apr 10, 2013 at 3:43 AM, A.Eibner <a_...@yahoo.de> wrote:
> : > >
> : > > > Hi Joel,
> : > > >
> : > > > I followed your steps, the cores and collection get created, but
> : > > > there is
> : > > > no leader elected so I can not query the collection...
> : > > > Do I miss something ?
> : > > >
> : > > > Kind Regards
> : > > > Alexander
> : > > >
> : > > > Am 2013-04-09 10:21, schrieb A.Eibner:
> : > > >
> : > > >   Hi,
> : > > > > thanks for your faster answer.
> : > > > >
> : > > > > You don't use the Collection API - may I ask you why ?
> : > > > > Therefore you have to setup everything (replicas, ...) manually...,
> : > > > > which I would like to avoid.
> : > > > >
> : > > > > Also what I don't understand, why my steps work in 4.0 but won't in
> : > > > > 4.2.1...
> : > > > > Any clues ?
> : > > > >
> : > > > > Kind Regards
> : > > > > Alexander
> : > > > >
> : > > > > Am 2013-04-08 19:12, schrieb Joel Bernstein:
> : > > > >
> : > > > > > The steps that I use to setup the collection are slightly different:
> : > > > > >
> : > > > > >
> : > > > > > 1) Start zk and upconfig the config set. Your approach is same.
> : > > > > > 2) Start appservers with Solr zkHost set to the zk started in step
> : > > > > > 1.
> : > > > > > 3) Use a core admin command to spin up a new core and collection.
> : > > > > >
> : > > > > >
> : > > > > > http://app01/solr/admin/cores?**action=CREATE&name=storage-**
> : > > > > > core&collection=storage&**numShards=1&collection.**
> : > > > > > configName=storage-conf<http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf>
> : > > > > >
> : > > > > > <http:/**/app03/solr/admin/collections?**action=CREATE&name=storage&**
> : > > > > > numShards=1&replicationFactor=**2&collection.configName=**storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
> : > > > > >
> : > > > > > >
> : > > > > >
> : > > > > > &shard=shard1
> : > > > > >
> : > > > > > This will spin up the new collection and initial core. I'm not using
> : > > > > > a
> : > > > > > replication factor because the following commands manually bind the
> : > > > > > replicas.
> : > > > > >
> : > > > > > 4) Spin up replica with a core admin command:
> : > > > > > http://app02/solr/admin/cores?**action=CREATE&name=storage-**
> : > > > > > core&collection=storage&<http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&>
> : > > > > >
> : > > > > > <http:**//app03/solr/admin/**collections?action=CREATE&**
> : > > > > > name=storage&numShards=1&**replicationFactor=2&**
> : > > > > > collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
> : > > > > >
> : > > > > > >
> : > > > > >
> : > > > > > shard=shard1
> : > > > > >
> : > > > > > 5) Same command as above on the 3rd server to spin up another
> : > > > > > replica.
> : > > > > >
> : > > > > > This will spin up a new core and bind it to shard1 of the storage
> : > > > > > collection.
> : > > > > >
> : > > > > >
> : > > > > >
> : > > > > >
> : > > > > >
> : > > > > > On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
> : > > > > >
> : > > > > >   Hi,
> : > > > > > >
> : > > > > > > I have a problem with setting up my solr cloud environment (on
> : > > > > > > three
> : > > > > > > machines).
> : > > > > > > If I want to create my collections from scratch I do the
> : > > > > > > following:
> : > > > > > >
> : > > > > > > *) Start ZooKeeper on all machines.
> : > > > > > >
> : > > > > > > *) Upload the configuration (on app02) for the collection via the
> : > > > > > > following command:
> : > > > > > >       zkcli.sh -cmd upconfig --zkhost
> : > > > > > > app01:4181,app02:4181,app03:****
> : > > > > > > 4181
> : > > > > > > --confdir config/solr/storage/conf/ --confname storage-conf
> : > > > > > >
> : > > > > > > *) Linking the configuration (on app02) via the following command:
> : > > > > > >       zkcli.sh -cmd linkconfig --collection storage --confname
> : > > > > > > storage-conf
> : > > > > > > --zkhost app01:4181,app02:4181,app03:****4181
> : > > > > > >
> : > > > > > > *) Start Tomcats (containing Solr) on app02,app03
> : > > > > > >
> : > > > > > > *) Create Collection via:
> : > > > > > > http://app03/solr/admin/****collections?action=CREATE&**<http://app03/solr/admin/**collections?action=CREATE&**>
> : > > > > > >
> : > > > > > > name=storage&numShards=1&****replicationFactor=2&**
> : > > > > > > collection.configName=storage-****conf<http://app03/solr/**
> : > > > > > > admin/collections?action=**CREATE&name=storage&numShards=**
> : > > > > > > 1&replicationFactor=2&**collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
> : > > > > > >
> : > > > > > > >
> : > > > > > >
> : > > > > > >
> : > > > > > > This creates the replication of the shard on app02 and app03, but
> : > > > > > > neither
> : > > > > > > of them is marked as leader, both are marked as DOWN.
> : > > > > > > And after wards I can not access the collection.
> : > > > > > > In the browser I get:
> : > > > > > > "SEVERE: org.apache.solr.common.****SolrException: no servers
> : > > > > > > hosting
> : > > > > > > shard:"
> : > > > > > >
> : > > > > > > In the log files the following error is present:
> : > > > > > > SEVERE: Error from shard: app02:9985/solr
> : > > > > > > org.apache.solr.common.****SolrException: Error CREATEing SolrCore
> : > > > > > > 'storage_shard1_replica1':
> : > > > > > >           at
> : > > > > > > org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
> : > > > > > > HttpSolrServer.java:404)
> : > > > > > >           at
> : > > > > > > org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
> : > > > > > > HttpSolrServer.java:181)
> : > > > > > >           at
> : > > > > > > org.apache.solr.handler.****component.HttpShardHandler$1.**
> : > > > > > > **
> : > > > > > > call(HttpShardHandler.java:****172)
> : > > > > > >           at
> : > > > > > > org.apache.solr.handler.****component.HttpShardHandler$1.**
> : > > > > > > **
> : > > > > > > call(HttpShardHandler.java:****135)
> : > > > > > >           at java.util.concurrent.****FutureTask$Sync.innerRun(**
> : > > > > > > FutureTask.java:334)
> : > > > > > >           at
> : > > > > > > java.util.concurrent.****FutureTask.run(FutureTask.****
> : > > > > > > java:166)
> : > > > > > >           at java.util.concurrent.****Executors$RunnableAdapter.**
> : > > > > > > call(Executors.java:471)
> : > > > > > >           at java.util.concurrent.****FutureTask$Sync.innerRun(**
> : > > > > > > FutureTask.java:334)
> : > > > > > >           at
> : > > > > > > java.util.concurrent.****FutureTask.run(FutureTask.****
> : > > > > > > java:166)
> : > > > > > >           at
> : > > > > > > java.util.concurrent.****ThreadPoolExecutor.runWorker(****
> : > > > > > > ThreadPoolExecutor.java:1110)
> : > > > > > >           at
> : > > > > > > java.util.concurrent.****ThreadPoolExecutor$Worker.run(****
> : > > > > > > ThreadPoolExecutor.java:603)
> : > > > > > >           at java.lang.Thread.run(Thread.****java:722)
> : > > > > > > Caused by: org.apache.solr.common.cloud.****ZooKeeperException:
> : > > > > > >           at
> : > > > > > > org.apache.solr.core.****CoreContainer.registerInZk(**
> : > > > > > > CoreContainer.java:922)
> : > > > > > >           at
> : > > > > > > org.apache.solr.core.****CoreContainer.registerCore(**
> : > > > > > > CoreContainer.java:892)
> : > > > > > >           at org.apache.solr.core.****CoreContainer.register(**
> : > > > > > > CoreContainer.java:841)
> : > > > > > >           at org.apache.solr.handler.admin.****CoreAdminHandler.**
> : > > > > > > handleCreateAction(****CoreAdminHandler.java:479)
> : > > > > > >           ... 19 more
> : > > > > > > Caused by: org.apache.solr.common.****SolrException: Error getting
> : > > > > > > leader
> : > > > > > > from zk for shard shard1
> : > > > > > >           at org.apache.solr.cloud.****ZkController.getLeader(**
> : > > > > > > ZkController.java:864)
> : > > > > > >           at org.apache.solr.cloud.****ZkController.register(**
> : > > > > > > ZkController.java:776)
> : > > > > > >           at org.apache.solr.cloud.****ZkController.register(**
> : > > > > > > ZkController.java:727)
> : > > > > > >           at
> : > > > > > > org.apache.solr.core.****CoreContainer.registerInZk(**
> : > > > > > > CoreContainer.java:908)
> : > > > > > >           ... 22 more
> : > > > > > > Caused by: java.lang.****InterruptedException: sleep interrupted
> : > > > > > >           at java.lang.Thread.sleep(Native Method)
> : > > > > > >           at
> : > > > > > > org.apache.solr.cloud.****ZkController.getLeaderProps(**
> : > > > > > > ZkController.java:905)
> : > > > > > >           at
> : > > > > > > org.apache.solr.cloud.****ZkController.getLeaderProps(**
> : > > > > > > ZkController.java:875)
> : > > > > > >           at org.apache.solr.cloud.****ZkController.getLeader(**
> : > > > > > > ZkController.java:839)
> : > > > > > >           ... 25 more
> : > > > > > >
> : > > > > > > I have attached a minimal set of configuration files which are
> : > > > > > > needed to
> : > > > > > > replicate this error, also containing the log files for the
> : > > > > > > commands
> : > > > > > > I have
> : > > > > > > run in the order above.
> : > > > > > >
> : > > > > > > I use the following versions of:
> : > > > > > > Solr: 4.2.1
> : > > > > > > ZooKeeper: 3.4.5
> : > > > > > > Tomcat 7.0.27
> : > > > > > >
> : > > > > > > app01: ZooKeeper
> : > > > > > > app02: ZooKeeper, Solr (in Tomcat)
> : > > > > > > app03: ZooKeeper, Solr (in Tomcat)
> : > > > > > >
> : > > > > > > The same procedure does work if I use Solr 4.0, but the BUG (
> : > > > > > > https://issues.apache.org/****jira/browse/SOLR-3939<https://issues.apache.org/**jira/browse/SOLR-3939>
> : > > > > > >
> : > > > > > > <https://**issues.apache.org/jira/browse/**SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
> : > > > > > >
> : > > > > > > >
> : > > > > > >
> : > > > > > > ) prevents me from using 4.0 in this scenario.
> : > > > > > >
> : > > > > > > If you need anything else, please just say so.
> : > > > > > >
> : > > > > > > Thanks for your help
> : > > > > > >
> : > > > > > > Kind Regards
> : > > > > > > Alexander
> : > > > > > >
> : > > > > > >
> : > > > > > >
> : > > > > >
> : > > > > >
> : > > > >
> : > > > >
> : > > >
> : > >
> : > >
> : >
> : >
> :
> :
>
> -Hoss
>


Re: Empty Solr 4.2.1 can not create Collection

Posted by Chris Hostetter <ho...@fucit.org>.
: sorry for pushing, but I just replayed the steps with solr 4.0 where
: everything works fine.
: Then I switched to solr 4.2.1 and replayed the exact same steps and the
: collection won't start and no leader will be elected.
: 
: Any clues ?
: Should I try it on the developer mailing list, maybe it's a bug ?

I'm not really understanding what the sequence of events is that's leading 
you to this error, but if you can reproduce a problem in which there is no 
leader election (and you get the NPE listed below) when creating a 
collection then yes, absolutely, please open a Jira and include...

1) the specific list of steps to reproduce starting from a 4.2.1 install
2) the configs you start with as well as any configs you are specifying 
when creating collections
3) snapshots of clusterstate.json taken before and after you encounter the 
problem
4) logs from each of hte solr servers you run in your test.



: 
: Kind Regards
: Alexander
: 
: Am 2013-04-10 22:27, schrieb A.Eibner:
: > Hi,
: > 
: > here the clusterstate.json (from zookeeper) after creating the core:
: > 
: > {"storage":{
: >      "shards":{"shard1":{
: >          "range":"80000000-7fffffff",
: >          "state":"active",
: >          "replicas":{"app02:9985_solr_storage-core":{
: >              "shard":"shard1",
: >              "state":"down",
: >              "core":"storage-core",
: >              "collection":"storage",
: >              "node_name":"app02:9985_solr",
: >              "base_url":"http://app02:9985/solr"}}}},
: >      "router":"compositeId"}}
: > cZxid = 0x100000024
: > ctime = Wed Apr 10 22:18:13 CEST 2013
: > mZxid = 0x10000003d
: > mtime = Wed Apr 10 22:21:26 CEST 2013
: > pZxid = 0x100000024
: > cversion = 0
: > dataVersion = 2
: > aclVersion = 0
: > ephemeralOwner = 0x0
: > dataLength = 467
: > numChildren = 0
: > 
: > But looking in the log files I found the following error (this also
: > occures with the collection api)
: > 
: > SEVERE: org.apache.solr.common.SolrException: Error CREATEing SolrCore
: > 'storage_shard1_replica1':
: >          at
: > org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:483)
: > 
: >          at
: > org.apache.solr.handler.admin.CoreAdminHandler.handleRequestBody(CoreAdminHandler.java:140)
: > 
: >          at
: > org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:135)
: > 
: >          at
: > org.apache.solr.servlet.SolrDispatchFilter.handleAdminRequest(SolrDispatchFilter.java:591)
: > 
: >          at
: > org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:192)
: > 
: >          at
: > org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:141)
: > 
: >          at
: > org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:243)
: > 
: >          at
: > org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:210)
: > 
: >          at
: > org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:225)
: > 
: >          at
: > org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:169)
: > 
: >          at
: > org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:168)
: > 
: >          at
: > org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:98)
: > 
: >          at
: > org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:118)
: > 
: >          at
: > org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:407)
: >          at
: > org.apache.coyote.http11.AbstractHttp11Processor.process(AbstractHttp11Processor.java:999)
: > 
: >          at
: > org.apache.coyote.AbstractProtocol$AbstractConnectionHandler.process(AbstractProtocol.java:565)
: > 
: >          at
: > org.apache.tomcat.util.net.JIoEndpoint$SocketProcessor.run(JIoEndpoint.java:307)
: > 
: >          at
: > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
: > 
: >          at
: > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
: > 
: >          at java.lang.Thread.run(Thread.java:722)
: > Caused by: org.apache.solr.common.cloud.ZooKeeperException:
: >          at
: > org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:931)
: >          at
: > org.apache.solr.core.CoreContainer.registerCore(CoreContainer.java:892)
: >          at
: > org.apache.solr.core.CoreContainer.register(CoreContainer.java:841)
: >          at
: > org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:479)
: > 
: >          ... 19 more
: > Caused by: java.lang.NullPointerException
: >          at
: > org.apache.solr.cloud.ShardLeaderElectionContext.runLeaderProcess(ElectionContext.java:190)
: > 
: >          at
: > org.apache.solr.cloud.LeaderElector.runIamLeaderProcess(LeaderElector.java:156)
: > 
: >          at
: > org.apache.solr.cloud.LeaderElector.checkIfIamLeader(LeaderElector.java:100)
: > 
: >          at
: > org.apache.solr.cloud.LeaderElector.joinElection(LeaderElector.java:266)
: >          at
: > org.apache.solr.cloud.ZkController.joinElection(ZkController.java:935)
: >          at
: > org.apache.solr.cloud.ZkController.register(ZkController.java:761)
: >          at
: > org.apache.solr.cloud.ZkController.register(ZkController.java:727)
: >          at
: > org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:908)
: >          ... 22 more
: > 
: > Kind regards
: > Alexander
: > 
: > Am 2013-04-10 19:12, schrieb Joel Bernstein:
: > > Can you post what your clusterstate.json?
: > > 
: > > After you spin up the initial core, it will automatically become
: > > leader for
: > > that shard.
: > > 
: > > 
: > > On Wed, Apr 10, 2013 at 3:43 AM, A.Eibner <a_...@yahoo.de> wrote:
: > > 
: > > > Hi Joel,
: > > > 
: > > > I followed your steps, the cores and collection get created, but
: > > > there is
: > > > no leader elected so I can not query the collection...
: > > > Do I miss something ?
: > > > 
: > > > Kind Regards
: > > > Alexander
: > > > 
: > > > Am 2013-04-09 10:21, schrieb A.Eibner:
: > > > 
: > > >   Hi,
: > > > > thanks for your faster answer.
: > > > > 
: > > > > You don't use the Collection API - may I ask you why ?
: > > > > Therefore you have to setup everything (replicas, ...) manually...,
: > > > > which I would like to avoid.
: > > > > 
: > > > > Also what I don't understand, why my steps work in 4.0 but won't in
: > > > > 4.2.1...
: > > > > Any clues ?
: > > > > 
: > > > > Kind Regards
: > > > > Alexander
: > > > > 
: > > > > Am 2013-04-08 19:12, schrieb Joel Bernstein:
: > > > > 
: > > > > > The steps that I use to setup the collection are slightly different:
: > > > > > 
: > > > > > 
: > > > > > 1) Start zk and upconfig the config set. Your approach is same.
: > > > > > 2) Start appservers with Solr zkHost set to the zk started in step
: > > > > > 1.
: > > > > > 3) Use a core admin command to spin up a new core and collection.
: > > > > > 
: > > > > > 
: > > > > > http://app01/solr/admin/cores?**action=CREATE&name=storage-**
: > > > > > core&collection=storage&**numShards=1&collection.**
: > > > > > configName=storage-conf<http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf>
: > > > > > 
: > > > > > <http:/**/app03/solr/admin/collections?**action=CREATE&name=storage&**
: > > > > > numShards=1&replicationFactor=**2&collection.configName=**storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
: > > > > > 
: > > > > > > 
: > > > > > 
: > > > > > &shard=shard1
: > > > > > 
: > > > > > This will spin up the new collection and initial core. I'm not using
: > > > > > a
: > > > > > replication factor because the following commands manually bind the
: > > > > > replicas.
: > > > > > 
: > > > > > 4) Spin up replica with a core admin command:
: > > > > > http://app02/solr/admin/cores?**action=CREATE&name=storage-**
: > > > > > core&collection=storage&<http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&>
: > > > > > 
: > > > > > <http:**//app03/solr/admin/**collections?action=CREATE&**
: > > > > > name=storage&numShards=1&**replicationFactor=2&**
: > > > > > collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
: > > > > > 
: > > > > > > 
: > > > > > 
: > > > > > shard=shard1
: > > > > > 
: > > > > > 5) Same command as above on the 3rd server to spin up another
: > > > > > replica.
: > > > > > 
: > > > > > This will spin up a new core and bind it to shard1 of the storage
: > > > > > collection.
: > > > > > 
: > > > > > 
: > > > > > 
: > > > > > 
: > > > > > 
: > > > > > On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
: > > > > > 
: > > > > >   Hi,
: > > > > > > 
: > > > > > > I have a problem with setting up my solr cloud environment (on
: > > > > > > three
: > > > > > > machines).
: > > > > > > If I want to create my collections from scratch I do the
: > > > > > > following:
: > > > > > > 
: > > > > > > *) Start ZooKeeper on all machines.
: > > > > > > 
: > > > > > > *) Upload the configuration (on app02) for the collection via the
: > > > > > > following command:
: > > > > > >       zkcli.sh -cmd upconfig --zkhost
: > > > > > > app01:4181,app02:4181,app03:****
: > > > > > > 4181
: > > > > > > --confdir config/solr/storage/conf/ --confname storage-conf
: > > > > > > 
: > > > > > > *) Linking the configuration (on app02) via the following command:
: > > > > > >       zkcli.sh -cmd linkconfig --collection storage --confname
: > > > > > > storage-conf
: > > > > > > --zkhost app01:4181,app02:4181,app03:****4181
: > > > > > > 
: > > > > > > *) Start Tomcats (containing Solr) on app02,app03
: > > > > > > 
: > > > > > > *) Create Collection via:
: > > > > > > http://app03/solr/admin/****collections?action=CREATE&**<http://app03/solr/admin/**collections?action=CREATE&**>
: > > > > > > 
: > > > > > > name=storage&numShards=1&****replicationFactor=2&**
: > > > > > > collection.configName=storage-****conf<http://app03/solr/**
: > > > > > > admin/collections?action=**CREATE&name=storage&numShards=**
: > > > > > > 1&replicationFactor=2&**collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
: > > > > > > 
: > > > > > > > 
: > > > > > > 
: > > > > > > 
: > > > > > > This creates the replication of the shard on app02 and app03, but
: > > > > > > neither
: > > > > > > of them is marked as leader, both are marked as DOWN.
: > > > > > > And after wards I can not access the collection.
: > > > > > > In the browser I get:
: > > > > > > "SEVERE: org.apache.solr.common.****SolrException: no servers
: > > > > > > hosting
: > > > > > > shard:"
: > > > > > > 
: > > > > > > In the log files the following error is present:
: > > > > > > SEVERE: Error from shard: app02:9985/solr
: > > > > > > org.apache.solr.common.****SolrException: Error CREATEing SolrCore
: > > > > > > 'storage_shard1_replica1':
: > > > > > >           at
: > > > > > > org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
: > > > > > > HttpSolrServer.java:404)
: > > > > > >           at
: > > > > > > org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
: > > > > > > HttpSolrServer.java:181)
: > > > > > >           at
: > > > > > > org.apache.solr.handler.****component.HttpShardHandler$1.**
: > > > > > > **
: > > > > > > call(HttpShardHandler.java:****172)
: > > > > > >           at
: > > > > > > org.apache.solr.handler.****component.HttpShardHandler$1.**
: > > > > > > **
: > > > > > > call(HttpShardHandler.java:****135)
: > > > > > >           at java.util.concurrent.****FutureTask$Sync.innerRun(**
: > > > > > > FutureTask.java:334)
: > > > > > >           at
: > > > > > > java.util.concurrent.****FutureTask.run(FutureTask.****
: > > > > > > java:166)
: > > > > > >           at java.util.concurrent.****Executors$RunnableAdapter.**
: > > > > > > call(Executors.java:471)
: > > > > > >           at java.util.concurrent.****FutureTask$Sync.innerRun(**
: > > > > > > FutureTask.java:334)
: > > > > > >           at
: > > > > > > java.util.concurrent.****FutureTask.run(FutureTask.****
: > > > > > > java:166)
: > > > > > >           at
: > > > > > > java.util.concurrent.****ThreadPoolExecutor.runWorker(****
: > > > > > > ThreadPoolExecutor.java:1110)
: > > > > > >           at
: > > > > > > java.util.concurrent.****ThreadPoolExecutor$Worker.run(****
: > > > > > > ThreadPoolExecutor.java:603)
: > > > > > >           at java.lang.Thread.run(Thread.****java:722)
: > > > > > > Caused by: org.apache.solr.common.cloud.****ZooKeeperException:
: > > > > > >           at
: > > > > > > org.apache.solr.core.****CoreContainer.registerInZk(**
: > > > > > > CoreContainer.java:922)
: > > > > > >           at
: > > > > > > org.apache.solr.core.****CoreContainer.registerCore(**
: > > > > > > CoreContainer.java:892)
: > > > > > >           at org.apache.solr.core.****CoreContainer.register(**
: > > > > > > CoreContainer.java:841)
: > > > > > >           at org.apache.solr.handler.admin.****CoreAdminHandler.**
: > > > > > > handleCreateAction(****CoreAdminHandler.java:479)
: > > > > > >           ... 19 more
: > > > > > > Caused by: org.apache.solr.common.****SolrException: Error getting
: > > > > > > leader
: > > > > > > from zk for shard shard1
: > > > > > >           at org.apache.solr.cloud.****ZkController.getLeader(**
: > > > > > > ZkController.java:864)
: > > > > > >           at org.apache.solr.cloud.****ZkController.register(**
: > > > > > > ZkController.java:776)
: > > > > > >           at org.apache.solr.cloud.****ZkController.register(**
: > > > > > > ZkController.java:727)
: > > > > > >           at
: > > > > > > org.apache.solr.core.****CoreContainer.registerInZk(**
: > > > > > > CoreContainer.java:908)
: > > > > > >           ... 22 more
: > > > > > > Caused by: java.lang.****InterruptedException: sleep interrupted
: > > > > > >           at java.lang.Thread.sleep(Native Method)
: > > > > > >           at
: > > > > > > org.apache.solr.cloud.****ZkController.getLeaderProps(**
: > > > > > > ZkController.java:905)
: > > > > > >           at
: > > > > > > org.apache.solr.cloud.****ZkController.getLeaderProps(**
: > > > > > > ZkController.java:875)
: > > > > > >           at org.apache.solr.cloud.****ZkController.getLeader(**
: > > > > > > ZkController.java:839)
: > > > > > >           ... 25 more
: > > > > > > 
: > > > > > > I have attached a minimal set of configuration files which are
: > > > > > > needed to
: > > > > > > replicate this error, also containing the log files for the
: > > > > > > commands
: > > > > > > I have
: > > > > > > run in the order above.
: > > > > > > 
: > > > > > > I use the following versions of:
: > > > > > > Solr: 4.2.1
: > > > > > > ZooKeeper: 3.4.5
: > > > > > > Tomcat 7.0.27
: > > > > > > 
: > > > > > > app01: ZooKeeper
: > > > > > > app02: ZooKeeper, Solr (in Tomcat)
: > > > > > > app03: ZooKeeper, Solr (in Tomcat)
: > > > > > > 
: > > > > > > The same procedure does work if I use Solr 4.0, but the BUG (
: > > > > > > https://issues.apache.org/****jira/browse/SOLR-3939<https://issues.apache.org/**jira/browse/SOLR-3939>
: > > > > > > 
: > > > > > > <https://**issues.apache.org/jira/browse/**SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
: > > > > > > 
: > > > > > > > 
: > > > > > > 
: > > > > > > ) prevents me from using 4.0 in this scenario.
: > > > > > > 
: > > > > > > If you need anything else, please just say so.
: > > > > > > 
: > > > > > > Thanks for your help
: > > > > > > 
: > > > > > > Kind Regards
: > > > > > > Alexander
: > > > > > > 
: > > > > > > 
: > > > > > > 
: > > > > > 
: > > > > > 
: > > > > 
: > > > > 
: > > > 
: > > 
: > > 
: > 
: > 
: 
: 

-Hoss

Re: Empty Solr 4.2.1 can not create Collection

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi,
sorry for pushing, but I just replayed the steps with solr 4.0 where 
everything works fine.
Then I switched to solr 4.2.1 and replayed the exact same steps and the 
collection won't start and no leader will be elected.

Any clues ?
Should I try it on the developer mailing list, maybe it's a bug ?

Kind Regards
Alexander

Am 2013-04-10 22:27, schrieb A.Eibner:
> Hi,
>
> here the clusterstate.json (from zookeeper) after creating the core:
>
> {"storage":{
>      "shards":{"shard1":{
>          "range":"80000000-7fffffff",
>          "state":"active",
>          "replicas":{"app02:9985_solr_storage-core":{
>              "shard":"shard1",
>              "state":"down",
>              "core":"storage-core",
>              "collection":"storage",
>              "node_name":"app02:9985_solr",
>              "base_url":"http://app02:9985/solr"}}}},
>      "router":"compositeId"}}
> cZxid = 0x100000024
> ctime = Wed Apr 10 22:18:13 CEST 2013
> mZxid = 0x10000003d
> mtime = Wed Apr 10 22:21:26 CEST 2013
> pZxid = 0x100000024
> cversion = 0
> dataVersion = 2
> aclVersion = 0
> ephemeralOwner = 0x0
> dataLength = 467
> numChildren = 0
>
> But looking in the log files I found the following error (this also
> occures with the collection api)
>
> SEVERE: org.apache.solr.common.SolrException: Error CREATEing SolrCore
> 'storage_shard1_replica1':
>          at
> org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:483)
>
>          at
> org.apache.solr.handler.admin.CoreAdminHandler.handleRequestBody(CoreAdminHandler.java:140)
>
>          at
> org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:135)
>
>          at
> org.apache.solr.servlet.SolrDispatchFilter.handleAdminRequest(SolrDispatchFilter.java:591)
>
>          at
> org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:192)
>
>          at
> org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:141)
>
>          at
> org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:243)
>
>          at
> org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:210)
>
>          at
> org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:225)
>
>          at
> org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:169)
>
>          at
> org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:168)
>
>          at
> org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:98)
>
>          at
> org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:118)
>
>          at
> org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:407)
>          at
> org.apache.coyote.http11.AbstractHttp11Processor.process(AbstractHttp11Processor.java:999)
>
>          at
> org.apache.coyote.AbstractProtocol$AbstractConnectionHandler.process(AbstractProtocol.java:565)
>
>          at
> org.apache.tomcat.util.net.JIoEndpoint$SocketProcessor.run(JIoEndpoint.java:307)
>
>          at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>
>          at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>
>          at java.lang.Thread.run(Thread.java:722)
> Caused by: org.apache.solr.common.cloud.ZooKeeperException:
>          at
> org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:931)
>          at
> org.apache.solr.core.CoreContainer.registerCore(CoreContainer.java:892)
>          at
> org.apache.solr.core.CoreContainer.register(CoreContainer.java:841)
>          at
> org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:479)
>
>          ... 19 more
> Caused by: java.lang.NullPointerException
>          at
> org.apache.solr.cloud.ShardLeaderElectionContext.runLeaderProcess(ElectionContext.java:190)
>
>          at
> org.apache.solr.cloud.LeaderElector.runIamLeaderProcess(LeaderElector.java:156)
>
>          at
> org.apache.solr.cloud.LeaderElector.checkIfIamLeader(LeaderElector.java:100)
>
>          at
> org.apache.solr.cloud.LeaderElector.joinElection(LeaderElector.java:266)
>          at
> org.apache.solr.cloud.ZkController.joinElection(ZkController.java:935)
>          at
> org.apache.solr.cloud.ZkController.register(ZkController.java:761)
>          at
> org.apache.solr.cloud.ZkController.register(ZkController.java:727)
>          at
> org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:908)
>          ... 22 more
>
> Kind regards
> Alexander
>
> Am 2013-04-10 19:12, schrieb Joel Bernstein:
>> Can you post what your clusterstate.json?
>>
>> After you spin up the initial core, it will automatically become
>> leader for
>> that shard.
>>
>>
>> On Wed, Apr 10, 2013 at 3:43 AM, A.Eibner <a_...@yahoo.de> wrote:
>>
>>> Hi Joel,
>>>
>>> I followed your steps, the cores and collection get created, but
>>> there is
>>> no leader elected so I can not query the collection...
>>> Do I miss something ?
>>>
>>> Kind Regards
>>> Alexander
>>>
>>> Am 2013-04-09 10:21, schrieb A.Eibner:
>>>
>>>   Hi,
>>>> thanks for your faster answer.
>>>>
>>>> You don't use the Collection API - may I ask you why ?
>>>> Therefore you have to setup everything (replicas, ...) manually...,
>>>> which I would like to avoid.
>>>>
>>>> Also what I don't understand, why my steps work in 4.0 but won't in
>>>> 4.2.1...
>>>> Any clues ?
>>>>
>>>> Kind Regards
>>>> Alexander
>>>>
>>>> Am 2013-04-08 19:12, schrieb Joel Bernstein:
>>>>
>>>>> The steps that I use to setup the collection are slightly different:
>>>>>
>>>>>
>>>>> 1) Start zk and upconfig the config set. Your approach is same.
>>>>> 2) Start appservers with Solr zkHost set to the zk started in step 1.
>>>>> 3) Use a core admin command to spin up a new core and collection.
>>>>>
>>>>>
>>>>> http://app01/solr/admin/cores?**action=CREATE&name=storage-**
>>>>> core&collection=storage&**numShards=1&collection.**
>>>>> configName=storage-conf<http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf>
>>>>>
>>>>> <http:/**/app03/solr/admin/collections?**action=CREATE&name=storage&**
>>>>> numShards=1&replicationFactor=**2&collection.configName=**storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>>>
>>>>>>
>>>>>
>>>>> &shard=shard1
>>>>>
>>>>> This will spin up the new collection and initial core. I'm not using a
>>>>> replication factor because the following commands manually bind the
>>>>> replicas.
>>>>>
>>>>> 4) Spin up replica with a core admin command:
>>>>> http://app02/solr/admin/cores?**action=CREATE&name=storage-**
>>>>> core&collection=storage&<http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&>
>>>>>
>>>>> <http:**//app03/solr/admin/**collections?action=CREATE&**
>>>>> name=storage&numShards=1&**replicationFactor=2&**
>>>>> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>>>
>>>>>>
>>>>>
>>>>> shard=shard1
>>>>>
>>>>> 5) Same command as above on the 3rd server to spin up another replica.
>>>>>
>>>>> This will spin up a new core and bind it to shard1 of the storage
>>>>> collection.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
>>>>>
>>>>>   Hi,
>>>>>>
>>>>>> I have a problem with setting up my solr cloud environment (on three
>>>>>> machines).
>>>>>> If I want to create my collections from scratch I do the following:
>>>>>>
>>>>>> *) Start ZooKeeper on all machines.
>>>>>>
>>>>>> *) Upload the configuration (on app02) for the collection via the
>>>>>> following command:
>>>>>>       zkcli.sh -cmd upconfig --zkhost
>>>>>> app01:4181,app02:4181,app03:****
>>>>>> 4181
>>>>>> --confdir config/solr/storage/conf/ --confname storage-conf
>>>>>>
>>>>>> *) Linking the configuration (on app02) via the following command:
>>>>>>       zkcli.sh -cmd linkconfig --collection storage --confname
>>>>>> storage-conf
>>>>>> --zkhost app01:4181,app02:4181,app03:****4181
>>>>>>
>>>>>> *) Start Tomcats (containing Solr) on app02,app03
>>>>>>
>>>>>> *) Create Collection via:
>>>>>> http://app03/solr/admin/****collections?action=CREATE&**<http://app03/solr/admin/**collections?action=CREATE&**>
>>>>>>
>>>>>> name=storage&numShards=1&****replicationFactor=2&**
>>>>>> collection.configName=storage-****conf<http://app03/solr/**
>>>>>> admin/collections?action=**CREATE&name=storage&numShards=**
>>>>>> 1&replicationFactor=2&**collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>> This creates the replication of the shard on app02 and app03, but
>>>>>> neither
>>>>>> of them is marked as leader, both are marked as DOWN.
>>>>>> And after wards I can not access the collection.
>>>>>> In the browser I get:
>>>>>> "SEVERE: org.apache.solr.common.****SolrException: no servers hosting
>>>>>> shard:"
>>>>>>
>>>>>> In the log files the following error is present:
>>>>>> SEVERE: Error from shard: app02:9985/solr
>>>>>> org.apache.solr.common.****SolrException: Error CREATEing SolrCore
>>>>>> 'storage_shard1_replica1':
>>>>>>           at
>>>>>> org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
>>>>>> HttpSolrServer.java:404)
>>>>>>           at
>>>>>> org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
>>>>>> HttpSolrServer.java:181)
>>>>>>           at
>>>>>> org.apache.solr.handler.****component.HttpShardHandler$1.**
>>>>>> **
>>>>>> call(HttpShardHandler.java:****172)
>>>>>>           at
>>>>>> org.apache.solr.handler.****component.HttpShardHandler$1.**
>>>>>> **
>>>>>> call(HttpShardHandler.java:****135)
>>>>>>           at java.util.concurrent.****FutureTask$Sync.innerRun(**
>>>>>> FutureTask.java:334)
>>>>>>           at java.util.concurrent.****FutureTask.run(FutureTask.****
>>>>>> java:166)
>>>>>>           at java.util.concurrent.****Executors$RunnableAdapter.**
>>>>>> call(Executors.java:471)
>>>>>>           at java.util.concurrent.****FutureTask$Sync.innerRun(**
>>>>>> FutureTask.java:334)
>>>>>>           at java.util.concurrent.****FutureTask.run(FutureTask.****
>>>>>> java:166)
>>>>>>           at
>>>>>> java.util.concurrent.****ThreadPoolExecutor.runWorker(****
>>>>>> ThreadPoolExecutor.java:1110)
>>>>>>           at
>>>>>> java.util.concurrent.****ThreadPoolExecutor$Worker.run(****
>>>>>> ThreadPoolExecutor.java:603)
>>>>>>           at java.lang.Thread.run(Thread.****java:722)
>>>>>> Caused by: org.apache.solr.common.cloud.****ZooKeeperException:
>>>>>>           at org.apache.solr.core.****CoreContainer.registerInZk(**
>>>>>> CoreContainer.java:922)
>>>>>>           at org.apache.solr.core.****CoreContainer.registerCore(**
>>>>>> CoreContainer.java:892)
>>>>>>           at org.apache.solr.core.****CoreContainer.register(**
>>>>>> CoreContainer.java:841)
>>>>>>           at org.apache.solr.handler.admin.****CoreAdminHandler.**
>>>>>> handleCreateAction(****CoreAdminHandler.java:479)
>>>>>>           ... 19 more
>>>>>> Caused by: org.apache.solr.common.****SolrException: Error getting
>>>>>> leader
>>>>>> from zk for shard shard1
>>>>>>           at org.apache.solr.cloud.****ZkController.getLeader(**
>>>>>> ZkController.java:864)
>>>>>>           at org.apache.solr.cloud.****ZkController.register(**
>>>>>> ZkController.java:776)
>>>>>>           at org.apache.solr.cloud.****ZkController.register(**
>>>>>> ZkController.java:727)
>>>>>>           at org.apache.solr.core.****CoreContainer.registerInZk(**
>>>>>> CoreContainer.java:908)
>>>>>>           ... 22 more
>>>>>> Caused by: java.lang.****InterruptedException: sleep interrupted
>>>>>>           at java.lang.Thread.sleep(Native Method)
>>>>>>           at org.apache.solr.cloud.****ZkController.getLeaderProps(**
>>>>>> ZkController.java:905)
>>>>>>           at org.apache.solr.cloud.****ZkController.getLeaderProps(**
>>>>>> ZkController.java:875)
>>>>>>           at org.apache.solr.cloud.****ZkController.getLeader(**
>>>>>> ZkController.java:839)
>>>>>>           ... 25 more
>>>>>>
>>>>>> I have attached a minimal set of configuration files which are
>>>>>> needed to
>>>>>> replicate this error, also containing the log files for the commands
>>>>>> I have
>>>>>> run in the order above.
>>>>>>
>>>>>> I use the following versions of:
>>>>>> Solr: 4.2.1
>>>>>> ZooKeeper: 3.4.5
>>>>>> Tomcat 7.0.27
>>>>>>
>>>>>> app01: ZooKeeper
>>>>>> app02: ZooKeeper, Solr (in Tomcat)
>>>>>> app03: ZooKeeper, Solr (in Tomcat)
>>>>>>
>>>>>> The same procedure does work if I use Solr 4.0, but the BUG (
>>>>>> https://issues.apache.org/****jira/browse/SOLR-3939<https://issues.apache.org/**jira/browse/SOLR-3939>
>>>>>>
>>>>>> <https://**issues.apache.org/jira/browse/**SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
>>>>>>
>>>>>>>
>>>>>>
>>>>>> ) prevents me from using 4.0 in this scenario.
>>>>>>
>>>>>> If you need anything else, please just say so.
>>>>>>
>>>>>> Thanks for your help
>>>>>>
>>>>>> Kind Regards
>>>>>> Alexander
>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>
>>
>>
>
>


Re: Empty Solr 4.2.1 can not create Collection

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi,

here the clusterstate.json (from zookeeper) after creating the core:

{"storage":{
     "shards":{"shard1":{
         "range":"80000000-7fffffff",
         "state":"active",
         "replicas":{"app02:9985_solr_storage-core":{
             "shard":"shard1",
             "state":"down",
             "core":"storage-core",
             "collection":"storage",
             "node_name":"app02:9985_solr",
             "base_url":"http://app02:9985/solr"}}}},
     "router":"compositeId"}}
cZxid = 0x100000024
ctime = Wed Apr 10 22:18:13 CEST 2013
mZxid = 0x10000003d
mtime = Wed Apr 10 22:21:26 CEST 2013
pZxid = 0x100000024
cversion = 0
dataVersion = 2
aclVersion = 0
ephemeralOwner = 0x0
dataLength = 467
numChildren = 0

But looking in the log files I found the following error (this also 
occures with the collection api)

SEVERE: org.apache.solr.common.SolrException: Error CREATEing SolrCore 
'storage_shard1_replica1':
         at 
org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:483)
         at 
org.apache.solr.handler.admin.CoreAdminHandler.handleRequestBody(CoreAdminHandler.java:140)
         at 
org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:135)
         at 
org.apache.solr.servlet.SolrDispatchFilter.handleAdminRequest(SolrDispatchFilter.java:591)
         at 
org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:192)
         at 
org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:141)
         at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:243)
         at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:210)
         at 
org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:225)
         at 
org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:169)
         at 
org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:168)
         at 
org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:98)
         at 
org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:118)
         at 
org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:407)
         at 
org.apache.coyote.http11.AbstractHttp11Processor.process(AbstractHttp11Processor.java:999)
         at 
org.apache.coyote.AbstractProtocol$AbstractConnectionHandler.process(AbstractProtocol.java:565)
         at 
org.apache.tomcat.util.net.JIoEndpoint$SocketProcessor.run(JIoEndpoint.java:307)
         at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
         at java.lang.Thread.run(Thread.java:722)
Caused by: org.apache.solr.common.cloud.ZooKeeperException:
         at 
org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:931)
         at 
org.apache.solr.core.CoreContainer.registerCore(CoreContainer.java:892)
         at 
org.apache.solr.core.CoreContainer.register(CoreContainer.java:841)
         at 
org.apache.solr.handler.admin.CoreAdminHandler.handleCreateAction(CoreAdminHandler.java:479)
         ... 19 more
Caused by: java.lang.NullPointerException
         at 
org.apache.solr.cloud.ShardLeaderElectionContext.runLeaderProcess(ElectionContext.java:190)
         at 
org.apache.solr.cloud.LeaderElector.runIamLeaderProcess(LeaderElector.java:156)
         at 
org.apache.solr.cloud.LeaderElector.checkIfIamLeader(LeaderElector.java:100)
         at 
org.apache.solr.cloud.LeaderElector.joinElection(LeaderElector.java:266)
         at 
org.apache.solr.cloud.ZkController.joinElection(ZkController.java:935)
         at 
org.apache.solr.cloud.ZkController.register(ZkController.java:761)
         at 
org.apache.solr.cloud.ZkController.register(ZkController.java:727)
         at 
org.apache.solr.core.CoreContainer.registerInZk(CoreContainer.java:908)
         ... 22 more

Kind regards
Alexander

Am 2013-04-10 19:12, schrieb Joel Bernstein:
> Can you post what your clusterstate.json?
>
> After you spin up the initial core, it will automatically become leader for
> that shard.
>
>
> On Wed, Apr 10, 2013 at 3:43 AM, A.Eibner <a_...@yahoo.de> wrote:
>
>> Hi Joel,
>>
>> I followed your steps, the cores and collection get created, but there is
>> no leader elected so I can not query the collection...
>> Do I miss something ?
>>
>> Kind Regards
>> Alexander
>>
>> Am 2013-04-09 10:21, schrieb A.Eibner:
>>
>>   Hi,
>>> thanks for your faster answer.
>>>
>>> You don't use the Collection API - may I ask you why ?
>>> Therefore you have to setup everything (replicas, ...) manually...,
>>> which I would like to avoid.
>>>
>>> Also what I don't understand, why my steps work in 4.0 but won't in
>>> 4.2.1...
>>> Any clues ?
>>>
>>> Kind Regards
>>> Alexander
>>>
>>> Am 2013-04-08 19:12, schrieb Joel Bernstein:
>>>
>>>> The steps that I use to setup the collection are slightly different:
>>>>
>>>>
>>>> 1) Start zk and upconfig the config set. Your approach is same.
>>>> 2) Start appservers with Solr zkHost set to the zk started in step 1.
>>>> 3) Use a core admin command to spin up a new core and collection.
>>>>
>>>>
>>>> http://app01/solr/admin/cores?**action=CREATE&name=storage-**
>>>> core&collection=storage&**numShards=1&collection.**
>>>> configName=storage-conf<http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf>
>>>> <http:/**/app03/solr/admin/collections?**action=CREATE&name=storage&**
>>>> numShards=1&replicationFactor=**2&collection.configName=**storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>>>
>>>>
>>>> &shard=shard1
>>>>
>>>> This will spin up the new collection and initial core. I'm not using a
>>>> replication factor because the following commands manually bind the
>>>> replicas.
>>>>
>>>> 4) Spin up replica with a core admin command:
>>>> http://app02/solr/admin/cores?**action=CREATE&name=storage-**
>>>> core&collection=storage&<http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&>
>>>> <http:**//app03/solr/admin/**collections?action=CREATE&**
>>>> name=storage&numShards=1&**replicationFactor=2&**
>>>> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>>>
>>>>
>>>> shard=shard1
>>>>
>>>> 5) Same command as above on the 3rd server to spin up another replica.
>>>>
>>>> This will spin up a new core and bind it to shard1 of the storage
>>>> collection.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
>>>>
>>>>   Hi,
>>>>>
>>>>> I have a problem with setting up my solr cloud environment (on three
>>>>> machines).
>>>>> If I want to create my collections from scratch I do the following:
>>>>>
>>>>> *) Start ZooKeeper on all machines.
>>>>>
>>>>> *) Upload the configuration (on app02) for the collection via the
>>>>> following command:
>>>>>       zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:****
>>>>> 4181
>>>>> --confdir config/solr/storage/conf/ --confname storage-conf
>>>>>
>>>>> *) Linking the configuration (on app02) via the following command:
>>>>>       zkcli.sh -cmd linkconfig --collection storage --confname
>>>>> storage-conf
>>>>> --zkhost app01:4181,app02:4181,app03:****4181
>>>>>
>>>>> *) Start Tomcats (containing Solr) on app02,app03
>>>>>
>>>>> *) Create Collection via:
>>>>> http://app03/solr/admin/****collections?action=CREATE&**<http://app03/solr/admin/**collections?action=CREATE&**>
>>>>> name=storage&numShards=1&****replicationFactor=2&**
>>>>> collection.configName=storage-****conf<http://app03/solr/**
>>>>> admin/collections?action=**CREATE&name=storage&numShards=**
>>>>> 1&replicationFactor=2&**collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>>>>
>>>>>
>>>>>
>>>>> This creates the replication of the shard on app02 and app03, but
>>>>> neither
>>>>> of them is marked as leader, both are marked as DOWN.
>>>>> And after wards I can not access the collection.
>>>>> In the browser I get:
>>>>> "SEVERE: org.apache.solr.common.****SolrException: no servers hosting
>>>>> shard:"
>>>>>
>>>>> In the log files the following error is present:
>>>>> SEVERE: Error from shard: app02:9985/solr
>>>>> org.apache.solr.common.****SolrException: Error CREATEing SolrCore
>>>>> 'storage_shard1_replica1':
>>>>>           at
>>>>> org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
>>>>> HttpSolrServer.java:404)
>>>>>           at
>>>>> org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
>>>>> HttpSolrServer.java:181)
>>>>>           at org.apache.solr.handler.****component.HttpShardHandler$1.**
>>>>> **
>>>>> call(HttpShardHandler.java:****172)
>>>>>           at org.apache.solr.handler.****component.HttpShardHandler$1.**
>>>>> **
>>>>> call(HttpShardHandler.java:****135)
>>>>>           at java.util.concurrent.****FutureTask$Sync.innerRun(**
>>>>> FutureTask.java:334)
>>>>>           at java.util.concurrent.****FutureTask.run(FutureTask.****
>>>>> java:166)
>>>>>           at java.util.concurrent.****Executors$RunnableAdapter.**
>>>>> call(Executors.java:471)
>>>>>           at java.util.concurrent.****FutureTask$Sync.innerRun(**
>>>>> FutureTask.java:334)
>>>>>           at java.util.concurrent.****FutureTask.run(FutureTask.****
>>>>> java:166)
>>>>>           at java.util.concurrent.****ThreadPoolExecutor.runWorker(****
>>>>> ThreadPoolExecutor.java:1110)
>>>>>           at java.util.concurrent.****ThreadPoolExecutor$Worker.run(****
>>>>> ThreadPoolExecutor.java:603)
>>>>>           at java.lang.Thread.run(Thread.****java:722)
>>>>> Caused by: org.apache.solr.common.cloud.****ZooKeeperException:
>>>>>           at org.apache.solr.core.****CoreContainer.registerInZk(**
>>>>> CoreContainer.java:922)
>>>>>           at org.apache.solr.core.****CoreContainer.registerCore(**
>>>>> CoreContainer.java:892)
>>>>>           at org.apache.solr.core.****CoreContainer.register(**
>>>>> CoreContainer.java:841)
>>>>>           at org.apache.solr.handler.admin.****CoreAdminHandler.**
>>>>> handleCreateAction(****CoreAdminHandler.java:479)
>>>>>           ... 19 more
>>>>> Caused by: org.apache.solr.common.****SolrException: Error getting
>>>>> leader
>>>>> from zk for shard shard1
>>>>>           at org.apache.solr.cloud.****ZkController.getLeader(**
>>>>> ZkController.java:864)
>>>>>           at org.apache.solr.cloud.****ZkController.register(**
>>>>> ZkController.java:776)
>>>>>           at org.apache.solr.cloud.****ZkController.register(**
>>>>> ZkController.java:727)
>>>>>           at org.apache.solr.core.****CoreContainer.registerInZk(**
>>>>> CoreContainer.java:908)
>>>>>           ... 22 more
>>>>> Caused by: java.lang.****InterruptedException: sleep interrupted
>>>>>           at java.lang.Thread.sleep(Native Method)
>>>>>           at org.apache.solr.cloud.****ZkController.getLeaderProps(**
>>>>> ZkController.java:905)
>>>>>           at org.apache.solr.cloud.****ZkController.getLeaderProps(**
>>>>> ZkController.java:875)
>>>>>           at org.apache.solr.cloud.****ZkController.getLeader(**
>>>>> ZkController.java:839)
>>>>>           ... 25 more
>>>>>
>>>>> I have attached a minimal set of configuration files which are needed to
>>>>> replicate this error, also containing the log files for the commands
>>>>> I have
>>>>> run in the order above.
>>>>>
>>>>> I use the following versions of:
>>>>> Solr: 4.2.1
>>>>> ZooKeeper: 3.4.5
>>>>> Tomcat 7.0.27
>>>>>
>>>>> app01: ZooKeeper
>>>>> app02: ZooKeeper, Solr (in Tomcat)
>>>>> app03: ZooKeeper, Solr (in Tomcat)
>>>>>
>>>>> The same procedure does work if I use Solr 4.0, but the BUG (
>>>>> https://issues.apache.org/****jira/browse/SOLR-3939<https://issues.apache.org/**jira/browse/SOLR-3939>
>>>>> <https://**issues.apache.org/jira/browse/**SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
>>>>>>
>>>>>
>>>>> ) prevents me from using 4.0 in this scenario.
>>>>>
>>>>> If you need anything else, please just say so.
>>>>>
>>>>> Thanks for your help
>>>>>
>>>>> Kind Regards
>>>>> Alexander
>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>
>>>
>>
>
>


Re: Empty Solr 4.2.1 can not create Collection

Posted by Joel Bernstein <jo...@gmail.com>.
Can you post what your clusterstate.json?

After you spin up the initial core, it will automatically become leader for
that shard.


On Wed, Apr 10, 2013 at 3:43 AM, A.Eibner <a_...@yahoo.de> wrote:

> Hi Joel,
>
> I followed your steps, the cores and collection get created, but there is
> no leader elected so I can not query the collection...
> Do I miss something ?
>
> Kind Regards
> Alexander
>
> Am 2013-04-09 10:21, schrieb A.Eibner:
>
>  Hi,
>> thanks for your faster answer.
>>
>> You don't use the Collection API - may I ask you why ?
>> Therefore you have to setup everything (replicas, ...) manually...,
>> which I would like to avoid.
>>
>> Also what I don't understand, why my steps work in 4.0 but won't in
>> 4.2.1...
>> Any clues ?
>>
>> Kind Regards
>> Alexander
>>
>> Am 2013-04-08 19:12, schrieb Joel Bernstein:
>>
>>> The steps that I use to setup the collection are slightly different:
>>>
>>>
>>> 1) Start zk and upconfig the config set. Your approach is same.
>>> 2) Start appservers with Solr zkHost set to the zk started in step 1.
>>> 3) Use a core admin command to spin up a new core and collection.
>>>
>>>
>>> http://app01/solr/admin/cores?**action=CREATE&name=storage-**
>>> core&collection=storage&**numShards=1&collection.**
>>> configName=storage-conf<http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf>
>>> <http:/**/app03/solr/admin/collections?**action=CREATE&name=storage&**
>>> numShards=1&replicationFactor=**2&collection.configName=**storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>> >
>>>
>>> &shard=shard1
>>>
>>> This will spin up the new collection and initial core. I'm not using a
>>> replication factor because the following commands manually bind the
>>> replicas.
>>>
>>> 4) Spin up replica with a core admin command:
>>> http://app02/solr/admin/cores?**action=CREATE&name=storage-**
>>> core&collection=storage&<http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&>
>>> <http:**//app03/solr/admin/**collections?action=CREATE&**
>>> name=storage&numShards=1&**replicationFactor=2&**
>>> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>> >
>>>
>>> shard=shard1
>>>
>>> 5) Same command as above on the 3rd server to spin up another replica.
>>>
>>> This will spin up a new core and bind it to shard1 of the storage
>>> collection.
>>>
>>>
>>>
>>>
>>>
>>> On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
>>>
>>>  Hi,
>>>>
>>>> I have a problem with setting up my solr cloud environment (on three
>>>> machines).
>>>> If I want to create my collections from scratch I do the following:
>>>>
>>>> *) Start ZooKeeper on all machines.
>>>>
>>>> *) Upload the configuration (on app02) for the collection via the
>>>> following command:
>>>>      zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:****
>>>> 4181
>>>> --confdir config/solr/storage/conf/ --confname storage-conf
>>>>
>>>> *) Linking the configuration (on app02) via the following command:
>>>>      zkcli.sh -cmd linkconfig --collection storage --confname
>>>> storage-conf
>>>> --zkhost app01:4181,app02:4181,app03:****4181
>>>>
>>>> *) Start Tomcats (containing Solr) on app02,app03
>>>>
>>>> *) Create Collection via:
>>>> http://app03/solr/admin/****collections?action=CREATE&**<http://app03/solr/admin/**collections?action=CREATE&**>
>>>> name=storage&numShards=1&****replicationFactor=2&**
>>>> collection.configName=storage-****conf<http://app03/solr/**
>>>> admin/collections?action=**CREATE&name=storage&numShards=**
>>>> 1&replicationFactor=2&**collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>> >
>>>>
>>>>
>>>> This creates the replication of the shard on app02 and app03, but
>>>> neither
>>>> of them is marked as leader, both are marked as DOWN.
>>>> And after wards I can not access the collection.
>>>> In the browser I get:
>>>> "SEVERE: org.apache.solr.common.****SolrException: no servers hosting
>>>> shard:"
>>>>
>>>> In the log files the following error is present:
>>>> SEVERE: Error from shard: app02:9985/solr
>>>> org.apache.solr.common.****SolrException: Error CREATEing SolrCore
>>>> 'storage_shard1_replica1':
>>>>          at
>>>> org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
>>>> HttpSolrServer.java:404)
>>>>          at
>>>> org.apache.solr.client.solrj.****impl.HttpSolrServer.request(****
>>>> HttpSolrServer.java:181)
>>>>          at org.apache.solr.handler.****component.HttpShardHandler$1.**
>>>> **
>>>> call(HttpShardHandler.java:****172)
>>>>          at org.apache.solr.handler.****component.HttpShardHandler$1.**
>>>> **
>>>> call(HttpShardHandler.java:****135)
>>>>          at java.util.concurrent.****FutureTask$Sync.innerRun(**
>>>> FutureTask.java:334)
>>>>          at java.util.concurrent.****FutureTask.run(FutureTask.****
>>>> java:166)
>>>>          at java.util.concurrent.****Executors$RunnableAdapter.**
>>>> call(Executors.java:471)
>>>>          at java.util.concurrent.****FutureTask$Sync.innerRun(**
>>>> FutureTask.java:334)
>>>>          at java.util.concurrent.****FutureTask.run(FutureTask.****
>>>> java:166)
>>>>          at java.util.concurrent.****ThreadPoolExecutor.runWorker(****
>>>> ThreadPoolExecutor.java:1110)
>>>>          at java.util.concurrent.****ThreadPoolExecutor$Worker.run(****
>>>> ThreadPoolExecutor.java:603)
>>>>          at java.lang.Thread.run(Thread.****java:722)
>>>> Caused by: org.apache.solr.common.cloud.****ZooKeeperException:
>>>>          at org.apache.solr.core.****CoreContainer.registerInZk(**
>>>> CoreContainer.java:922)
>>>>          at org.apache.solr.core.****CoreContainer.registerCore(**
>>>> CoreContainer.java:892)
>>>>          at org.apache.solr.core.****CoreContainer.register(**
>>>> CoreContainer.java:841)
>>>>          at org.apache.solr.handler.admin.****CoreAdminHandler.**
>>>> handleCreateAction(****CoreAdminHandler.java:479)
>>>>          ... 19 more
>>>> Caused by: org.apache.solr.common.****SolrException: Error getting
>>>> leader
>>>> from zk for shard shard1
>>>>          at org.apache.solr.cloud.****ZkController.getLeader(**
>>>> ZkController.java:864)
>>>>          at org.apache.solr.cloud.****ZkController.register(**
>>>> ZkController.java:776)
>>>>          at org.apache.solr.cloud.****ZkController.register(**
>>>> ZkController.java:727)
>>>>          at org.apache.solr.core.****CoreContainer.registerInZk(**
>>>> CoreContainer.java:908)
>>>>          ... 22 more
>>>> Caused by: java.lang.****InterruptedException: sleep interrupted
>>>>          at java.lang.Thread.sleep(Native Method)
>>>>          at org.apache.solr.cloud.****ZkController.getLeaderProps(**
>>>> ZkController.java:905)
>>>>          at org.apache.solr.cloud.****ZkController.getLeaderProps(**
>>>> ZkController.java:875)
>>>>          at org.apache.solr.cloud.****ZkController.getLeader(**
>>>> ZkController.java:839)
>>>>          ... 25 more
>>>>
>>>> I have attached a minimal set of configuration files which are needed to
>>>> replicate this error, also containing the log files for the commands
>>>> I have
>>>> run in the order above.
>>>>
>>>> I use the following versions of:
>>>> Solr: 4.2.1
>>>> ZooKeeper: 3.4.5
>>>> Tomcat 7.0.27
>>>>
>>>> app01: ZooKeeper
>>>> app02: ZooKeeper, Solr (in Tomcat)
>>>> app03: ZooKeeper, Solr (in Tomcat)
>>>>
>>>> The same procedure does work if I use Solr 4.0, but the BUG (
>>>> https://issues.apache.org/****jira/browse/SOLR-3939<https://issues.apache.org/**jira/browse/SOLR-3939>
>>>> <https://**issues.apache.org/jira/browse/**SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
>>>> >
>>>>
>>>> ) prevents me from using 4.0 in this scenario.
>>>>
>>>> If you need anything else, please just say so.
>>>>
>>>> Thanks for your help
>>>>
>>>> Kind Regards
>>>> Alexander
>>>>
>>>>
>>>>
>>>
>>>
>>
>>
>


-- 
Joel Bernstein
Professional Services LucidWorks

Re: Empty Solr 4.2.1 can not create Collection

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi Joel,

I followed your steps, the cores and collection get created, but there 
is no leader elected so I can not query the collection...
Do I miss something ?

Kind Regards
Alexander

Am 2013-04-09 10:21, schrieb A.Eibner:
> Hi,
> thanks for your faster answer.
>
> You don't use the Collection API - may I ask you why ?
> Therefore you have to setup everything (replicas, ...) manually...,
> which I would like to avoid.
>
> Also what I don't understand, why my steps work in 4.0 but won't in
> 4.2.1...
> Any clues ?
>
> Kind Regards
> Alexander
>
> Am 2013-04-08 19:12, schrieb Joel Bernstein:
>> The steps that I use to setup the collection are slightly different:
>>
>>
>> 1) Start zk and upconfig the config set. Your approach is same.
>> 2) Start appservers with Solr zkHost set to the zk started in step 1.
>> 3) Use a core admin command to spin up a new core and collection.
>>
>>
>> http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>
>> &shard=shard1
>>
>> This will spin up the new collection and initial core. I'm not using a
>> replication factor because the following commands manually bind the
>> replicas.
>>
>> 4) Spin up replica with a core admin command:
>> http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>
>> shard=shard1
>>
>> 5) Same command as above on the 3rd server to spin up another replica.
>>
>> This will spin up a new core and bind it to shard1 of the storage
>> collection.
>>
>>
>>
>>
>>
>> On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
>>
>>> Hi,
>>>
>>> I have a problem with setting up my solr cloud environment (on three
>>> machines).
>>> If I want to create my collections from scratch I do the following:
>>>
>>> *) Start ZooKeeper on all machines.
>>>
>>> *) Upload the configuration (on app02) for the collection via the
>>> following command:
>>>      zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:**4181
>>> --confdir config/solr/storage/conf/ --confname storage-conf
>>>
>>> *) Linking the configuration (on app02) via the following command:
>>>      zkcli.sh -cmd linkconfig --collection storage --confname
>>> storage-conf
>>> --zkhost app01:4181,app02:4181,app03:**4181
>>>
>>> *) Start Tomcats (containing Solr) on app02,app03
>>>
>>> *) Create Collection via:
>>> http://app03/solr/admin/**collections?action=CREATE&**
>>> name=storage&numShards=1&**replicationFactor=2&**
>>> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>
>>>
>>> This creates the replication of the shard on app02 and app03, but
>>> neither
>>> of them is marked as leader, both are marked as DOWN.
>>> And after wards I can not access the collection.
>>> In the browser I get:
>>> "SEVERE: org.apache.solr.common.**SolrException: no servers hosting
>>> shard:"
>>>
>>> In the log files the following error is present:
>>> SEVERE: Error from shard: app02:9985/solr
>>> org.apache.solr.common.**SolrException: Error CREATEing SolrCore
>>> 'storage_shard1_replica1':
>>>          at
>>> org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>>> HttpSolrServer.java:404)
>>>          at
>>> org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>>> HttpSolrServer.java:181)
>>>          at org.apache.solr.handler.**component.HttpShardHandler$1.**
>>> call(HttpShardHandler.java:**172)
>>>          at org.apache.solr.handler.**component.HttpShardHandler$1.**
>>> call(HttpShardHandler.java:**135)
>>>          at java.util.concurrent.**FutureTask$Sync.innerRun(**
>>> FutureTask.java:334)
>>>          at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>>          at java.util.concurrent.**Executors$RunnableAdapter.**
>>> call(Executors.java:471)
>>>          at java.util.concurrent.**FutureTask$Sync.innerRun(**
>>> FutureTask.java:334)
>>>          at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>>          at java.util.concurrent.**ThreadPoolExecutor.runWorker(**
>>> ThreadPoolExecutor.java:1110)
>>>          at java.util.concurrent.**ThreadPoolExecutor$Worker.run(**
>>> ThreadPoolExecutor.java:603)
>>>          at java.lang.Thread.run(Thread.**java:722)
>>> Caused by: org.apache.solr.common.cloud.**ZooKeeperException:
>>>          at org.apache.solr.core.**CoreContainer.registerInZk(**
>>> CoreContainer.java:922)
>>>          at org.apache.solr.core.**CoreContainer.registerCore(**
>>> CoreContainer.java:892)
>>>          at org.apache.solr.core.**CoreContainer.register(**
>>> CoreContainer.java:841)
>>>          at org.apache.solr.handler.admin.**CoreAdminHandler.**
>>> handleCreateAction(**CoreAdminHandler.java:479)
>>>          ... 19 more
>>> Caused by: org.apache.solr.common.**SolrException: Error getting leader
>>> from zk for shard shard1
>>>          at org.apache.solr.cloud.**ZkController.getLeader(**
>>> ZkController.java:864)
>>>          at org.apache.solr.cloud.**ZkController.register(**
>>> ZkController.java:776)
>>>          at org.apache.solr.cloud.**ZkController.register(**
>>> ZkController.java:727)
>>>          at org.apache.solr.core.**CoreContainer.registerInZk(**
>>> CoreContainer.java:908)
>>>          ... 22 more
>>> Caused by: java.lang.**InterruptedException: sleep interrupted
>>>          at java.lang.Thread.sleep(Native Method)
>>>          at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>>> ZkController.java:905)
>>>          at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>>> ZkController.java:875)
>>>          at org.apache.solr.cloud.**ZkController.getLeader(**
>>> ZkController.java:839)
>>>          ... 25 more
>>>
>>> I have attached a minimal set of configuration files which are needed to
>>> replicate this error, also containing the log files for the commands
>>> I have
>>> run in the order above.
>>>
>>> I use the following versions of:
>>> Solr: 4.2.1
>>> ZooKeeper: 3.4.5
>>> Tomcat 7.0.27
>>>
>>> app01: ZooKeeper
>>> app02: ZooKeeper, Solr (in Tomcat)
>>> app03: ZooKeeper, Solr (in Tomcat)
>>>
>>> The same procedure does work if I use Solr 4.0, but the BUG (
>>> https://issues.apache.org/**jira/browse/SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
>>>
>>> ) prevents me from using 4.0 in this scenario.
>>>
>>> If you need anything else, please just say so.
>>>
>>> Thanks for your help
>>>
>>> Kind Regards
>>> Alexander
>>>
>>>
>>
>>
>
>


Re: Empty Solr 4.2.1 can not create Collection

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi,
thanks for your faster answer.

You don't use the Collection API - may I ask you why ?
Therefore you have to setup everything (replicas, ...) manually..., 
which I would like to avoid.

Also what I don't understand, why my steps work in 4.0 but won't in 4.2.1...
Any clues ?

Kind Regards
Alexander

Am 2013-04-08 19:12, schrieb Joel Bernstein:
> The steps that I use to setup the collection are slightly different:
>
>
> 1) Start zk and upconfig the config set. Your approach is same.
> 2) Start appservers with Solr zkHost set to the zk started in step 1.
> 3) Use a core admin command to spin up a new core and collection.
>
>
> http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
> &shard=shard1
>
> This will spin up the new collection and initial core. I'm not using a
> replication factor because the following commands manually bind the
> replicas.
>
> 4) Spin up replica with a core admin command:
> http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
> shard=shard1
>
> 5) Same command as above on the 3rd server to spin up another replica.
>
> This will spin up a new core and bind it to shard1 of the storage
> collection.
>
>
>
>
>
> On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
>
>> Hi,
>>
>> I have a problem with setting up my solr cloud environment (on three
>> machines).
>> If I want to create my collections from scratch I do the following:
>>
>> *) Start ZooKeeper on all machines.
>>
>> *) Upload the configuration (on app02) for the collection via the
>> following command:
>>      zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:**4181
>> --confdir config/solr/storage/conf/ --confname storage-conf
>>
>> *) Linking the configuration (on app02) via the following command:
>>      zkcli.sh -cmd linkconfig --collection storage --confname storage-conf
>> --zkhost app01:4181,app02:4181,app03:**4181
>>
>> *) Start Tomcats (containing Solr) on app02,app03
>>
>> *) Create Collection via:
>> http://app03/solr/admin/**collections?action=CREATE&**
>> name=storage&numShards=1&**replicationFactor=2&**
>> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>
>> This creates the replication of the shard on app02 and app03, but neither
>> of them is marked as leader, both are marked as DOWN.
>> And after wards I can not access the collection.
>> In the browser I get:
>> "SEVERE: org.apache.solr.common.**SolrException: no servers hosting
>> shard:"
>>
>> In the log files the following error is present:
>> SEVERE: Error from shard: app02:9985/solr
>> org.apache.solr.common.**SolrException: Error CREATEing SolrCore
>> 'storage_shard1_replica1':
>>          at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>> HttpSolrServer.java:404)
>>          at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>> HttpSolrServer.java:181)
>>          at org.apache.solr.handler.**component.HttpShardHandler$1.**
>> call(HttpShardHandler.java:**172)
>>          at org.apache.solr.handler.**component.HttpShardHandler$1.**
>> call(HttpShardHandler.java:**135)
>>          at java.util.concurrent.**FutureTask$Sync.innerRun(**
>> FutureTask.java:334)
>>          at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>          at java.util.concurrent.**Executors$RunnableAdapter.**
>> call(Executors.java:471)
>>          at java.util.concurrent.**FutureTask$Sync.innerRun(**
>> FutureTask.java:334)
>>          at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>          at java.util.concurrent.**ThreadPoolExecutor.runWorker(**
>> ThreadPoolExecutor.java:1110)
>>          at java.util.concurrent.**ThreadPoolExecutor$Worker.run(**
>> ThreadPoolExecutor.java:603)
>>          at java.lang.Thread.run(Thread.**java:722)
>> Caused by: org.apache.solr.common.cloud.**ZooKeeperException:
>>          at org.apache.solr.core.**CoreContainer.registerInZk(**
>> CoreContainer.java:922)
>>          at org.apache.solr.core.**CoreContainer.registerCore(**
>> CoreContainer.java:892)
>>          at org.apache.solr.core.**CoreContainer.register(**
>> CoreContainer.java:841)
>>          at org.apache.solr.handler.admin.**CoreAdminHandler.**
>> handleCreateAction(**CoreAdminHandler.java:479)
>>          ... 19 more
>> Caused by: org.apache.solr.common.**SolrException: Error getting leader
>> from zk for shard shard1
>>          at org.apache.solr.cloud.**ZkController.getLeader(**
>> ZkController.java:864)
>>          at org.apache.solr.cloud.**ZkController.register(**
>> ZkController.java:776)
>>          at org.apache.solr.cloud.**ZkController.register(**
>> ZkController.java:727)
>>          at org.apache.solr.core.**CoreContainer.registerInZk(**
>> CoreContainer.java:908)
>>          ... 22 more
>> Caused by: java.lang.**InterruptedException: sleep interrupted
>>          at java.lang.Thread.sleep(Native Method)
>>          at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>> ZkController.java:905)
>>          at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>> ZkController.java:875)
>>          at org.apache.solr.cloud.**ZkController.getLeader(**
>> ZkController.java:839)
>>          ... 25 more
>>
>> I have attached a minimal set of configuration files which are needed to
>> replicate this error, also containing the log files for the commands I have
>> run in the order above.
>>
>> I use the following versions of:
>> Solr: 4.2.1
>> ZooKeeper: 3.4.5
>> Tomcat 7.0.27
>>
>> app01: ZooKeeper
>> app02: ZooKeeper, Solr (in Tomcat)
>> app03: ZooKeeper, Solr (in Tomcat)
>>
>> The same procedure does work if I use Solr 4.0, but the BUG (
>> https://issues.apache.org/**jira/browse/SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
>> ) prevents me from using 4.0 in this scenario.
>>
>> If you need anything else, please just say so.
>>
>> Thanks for your help
>>
>> Kind Regards
>> Alexander
>>
>>
>
>


Re: Empty Solr 4.2.1 can not create Collection

Posted by "A.Eibner" <a_...@yahoo.de>.
Hi,

you are right, I have removed "collection1" from the solr.xml but set 
defaultCoreName="storage".

Also this works in 4.0 but won't in 4.2.1, any clues ?

Kind Regards
Alexander

Am 2013-04-08 20:06, schrieb Joel Bernstein:
> The scenario above needs to have collection1 removed from the solr.xml to
> work. This, I believe, is the "Empty Solr" scenario that you are talking
> about. If you don't remove collection1 from solr.xml on all the solr
> instances, they will get tripped up on collection1 during these steps.
>
> If you startup with collection1 in solr.xml it's best to startup the
> initial Solr instance with the bootstrap-conf parameter so Solr can
> properly create this collection.
>
>
> On Mon, Apr 8, 2013 at 1:12 PM, Joel Bernstein <jo...@gmail.com> wrote:
>
>> The steps that I use to setup the collection are slightly different:
>>
>>
>> 1) Start zk and upconfig the config set. Your approach is same.
>> 2) Start appservers with Solr zkHost set to the zk started in step 1.
>> 3) Use a core admin command to spin up a new core and collection.
>>
>>
>> http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>> &shard=shard1
>>
>> This will spin up the new collection and initial core. I'm not using a
>> replication factor because the following commands manually bind the
>> replicas.
>>
>> 4) Spin up replica with a core admin command:
>>
>> http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>> shard=shard1
>>
>> 5) Same command as above on the 3rd server to spin up another replica.
>>
>> This will spin up a new core and bind it to shard1 of the storage
>> collection.
>>
>>
>>
>>
>>
>> On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
>>
>>> Hi,
>>>
>>> I have a problem with setting up my solr cloud environment (on three
>>> machines).
>>> If I want to create my collections from scratch I do the following:
>>>
>>> *) Start ZooKeeper on all machines.
>>>
>>> *) Upload the configuration (on app02) for the collection via the
>>> following command:
>>>      zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:**4181
>>> --confdir config/solr/storage/conf/ --confname storage-conf
>>>
>>> *) Linking the configuration (on app02) via the following command:
>>>      zkcli.sh -cmd linkconfig --collection storage --confname storage-conf
>>> --zkhost app01:4181,app02:4181,app03:**4181
>>>
>>> *) Start Tomcats (containing Solr) on app02,app03
>>>
>>> *) Create Collection via:
>>> http://app03/solr/admin/**collections?action=CREATE&**
>>> name=storage&numShards=1&**replicationFactor=2&**
>>> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>>
>>> This creates the replication of the shard on app02 and app03, but neither
>>> of them is marked as leader, both are marked as DOWN.
>>> And after wards I can not access the collection.
>>> In the browser I get:
>>> "SEVERE: org.apache.solr.common.**SolrException: no servers hosting
>>> shard:"
>>>
>>> In the log files the following error is present:
>>> SEVERE: Error from shard: app02:9985/solr
>>> org.apache.solr.common.**SolrException: Error CREATEing SolrCore
>>> 'storage_shard1_replica1':
>>>          at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>>> HttpSolrServer.java:404)
>>>          at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>>> HttpSolrServer.java:181)
>>>          at org.apache.solr.handler.**component.HttpShardHandler$1.**
>>> call(HttpShardHandler.java:**172)
>>>          at org.apache.solr.handler.**component.HttpShardHandler$1.**
>>> call(HttpShardHandler.java:**135)
>>>          at java.util.concurrent.**FutureTask$Sync.innerRun(**
>>> FutureTask.java:334)
>>>          at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>>          at java.util.concurrent.**Executors$RunnableAdapter.**
>>> call(Executors.java:471)
>>>          at java.util.concurrent.**FutureTask$Sync.innerRun(**
>>> FutureTask.java:334)
>>>          at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>>          at java.util.concurrent.**ThreadPoolExecutor.runWorker(**
>>> ThreadPoolExecutor.java:1110)
>>>          at java.util.concurrent.**ThreadPoolExecutor$Worker.run(**
>>> ThreadPoolExecutor.java:603)
>>>          at java.lang.Thread.run(Thread.**java:722)
>>> Caused by: org.apache.solr.common.cloud.**ZooKeeperException:
>>>          at org.apache.solr.core.**CoreContainer.registerInZk(**
>>> CoreContainer.java:922)
>>>          at org.apache.solr.core.**CoreContainer.registerCore(**
>>> CoreContainer.java:892)
>>>          at org.apache.solr.core.**CoreContainer.register(**
>>> CoreContainer.java:841)
>>>          at org.apache.solr.handler.admin.**CoreAdminHandler.**
>>> handleCreateAction(**CoreAdminHandler.java:479)
>>>          ... 19 more
>>> Caused by: org.apache.solr.common.**SolrException: Error getting leader
>>> from zk for shard shard1
>>>          at org.apache.solr.cloud.**ZkController.getLeader(**
>>> ZkController.java:864)
>>>          at org.apache.solr.cloud.**ZkController.register(**
>>> ZkController.java:776)
>>>          at org.apache.solr.cloud.**ZkController.register(**
>>> ZkController.java:727)
>>>          at org.apache.solr.core.**CoreContainer.registerInZk(**
>>> CoreContainer.java:908)
>>>          ... 22 more
>>> Caused by: java.lang.**InterruptedException: sleep interrupted
>>>          at java.lang.Thread.sleep(Native Method)
>>>          at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>>> ZkController.java:905)
>>>          at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>>> ZkController.java:875)
>>>          at org.apache.solr.cloud.**ZkController.getLeader(**
>>> ZkController.java:839)
>>>          ... 25 more
>>>
>>> I have attached a minimal set of configuration files which are needed to
>>> replicate this error, also containing the log files for the commands I have
>>> run in the order above.
>>>
>>> I use the following versions of:
>>> Solr: 4.2.1
>>> ZooKeeper: 3.4.5
>>> Tomcat 7.0.27
>>>
>>> app01: ZooKeeper
>>> app02: ZooKeeper, Solr (in Tomcat)
>>> app03: ZooKeeper, Solr (in Tomcat)
>>>
>>> The same procedure does work if I use Solr 4.0, but the BUG (
>>> https://issues.apache.org/**jira/browse/SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
>>> ) prevents me from using 4.0 in this scenario.
>>>
>>> If you need anything else, please just say so.
>>>
>>> Thanks for your help
>>>
>>> Kind Regards
>>> Alexander
>>>
>>>
>>
>>
>> --
>> Joel Bernstein
>> Professional Services LucidWorks
>>
>
>
>


Re: Empty Solr 4.2.1 can not create Collection

Posted by Joel Bernstein <jo...@gmail.com>.
The scenario above needs to have collection1 removed from the solr.xml to
work. This, I believe, is the "Empty Solr" scenario that you are talking
about. If you don't remove collection1 from solr.xml on all the solr
instances, they will get tripped up on collection1 during these steps.

If you startup with collection1 in solr.xml it's best to startup the
initial Solr instance with the bootstrap-conf parameter so Solr can
properly create this collection.


On Mon, Apr 8, 2013 at 1:12 PM, Joel Bernstein <jo...@gmail.com> wrote:

> The steps that I use to setup the collection are slightly different:
>
>
> 1) Start zk and upconfig the config set. Your approach is same.
> 2) Start appservers with Solr zkHost set to the zk started in step 1.
> 3) Use a core admin command to spin up a new core and collection.
>
>
> http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
> &shard=shard1
>
> This will spin up the new collection and initial core. I'm not using a
> replication factor because the following commands manually bind the
> replicas.
>
> 4) Spin up replica with a core admin command:
>
> http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
> shard=shard1
>
> 5) Same command as above on the 3rd server to spin up another replica.
>
> This will spin up a new core and bind it to shard1 of the storage
> collection.
>
>
>
>
>
> On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:
>
>> Hi,
>>
>> I have a problem with setting up my solr cloud environment (on three
>> machines).
>> If I want to create my collections from scratch I do the following:
>>
>> *) Start ZooKeeper on all machines.
>>
>> *) Upload the configuration (on app02) for the collection via the
>> following command:
>>     zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:**4181
>> --confdir config/solr/storage/conf/ --confname storage-conf
>>
>> *) Linking the configuration (on app02) via the following command:
>>     zkcli.sh -cmd linkconfig --collection storage --confname storage-conf
>> --zkhost app01:4181,app02:4181,app03:**4181
>>
>> *) Start Tomcats (containing Solr) on app02,app03
>>
>> *) Create Collection via:
>> http://app03/solr/admin/**collections?action=CREATE&**
>> name=storage&numShards=1&**replicationFactor=2&**
>> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>>
>> This creates the replication of the shard on app02 and app03, but neither
>> of them is marked as leader, both are marked as DOWN.
>> And after wards I can not access the collection.
>> In the browser I get:
>> "SEVERE: org.apache.solr.common.**SolrException: no servers hosting
>> shard:"
>>
>> In the log files the following error is present:
>> SEVERE: Error from shard: app02:9985/solr
>> org.apache.solr.common.**SolrException: Error CREATEing SolrCore
>> 'storage_shard1_replica1':
>>         at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>> HttpSolrServer.java:404)
>>         at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
>> HttpSolrServer.java:181)
>>         at org.apache.solr.handler.**component.HttpShardHandler$1.**
>> call(HttpShardHandler.java:**172)
>>         at org.apache.solr.handler.**component.HttpShardHandler$1.**
>> call(HttpShardHandler.java:**135)
>>         at java.util.concurrent.**FutureTask$Sync.innerRun(**
>> FutureTask.java:334)
>>         at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>         at java.util.concurrent.**Executors$RunnableAdapter.**
>> call(Executors.java:471)
>>         at java.util.concurrent.**FutureTask$Sync.innerRun(**
>> FutureTask.java:334)
>>         at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>>         at java.util.concurrent.**ThreadPoolExecutor.runWorker(**
>> ThreadPoolExecutor.java:1110)
>>         at java.util.concurrent.**ThreadPoolExecutor$Worker.run(**
>> ThreadPoolExecutor.java:603)
>>         at java.lang.Thread.run(Thread.**java:722)
>> Caused by: org.apache.solr.common.cloud.**ZooKeeperException:
>>         at org.apache.solr.core.**CoreContainer.registerInZk(**
>> CoreContainer.java:922)
>>         at org.apache.solr.core.**CoreContainer.registerCore(**
>> CoreContainer.java:892)
>>         at org.apache.solr.core.**CoreContainer.register(**
>> CoreContainer.java:841)
>>         at org.apache.solr.handler.admin.**CoreAdminHandler.**
>> handleCreateAction(**CoreAdminHandler.java:479)
>>         ... 19 more
>> Caused by: org.apache.solr.common.**SolrException: Error getting leader
>> from zk for shard shard1
>>         at org.apache.solr.cloud.**ZkController.getLeader(**
>> ZkController.java:864)
>>         at org.apache.solr.cloud.**ZkController.register(**
>> ZkController.java:776)
>>         at org.apache.solr.cloud.**ZkController.register(**
>> ZkController.java:727)
>>         at org.apache.solr.core.**CoreContainer.registerInZk(**
>> CoreContainer.java:908)
>>         ... 22 more
>> Caused by: java.lang.**InterruptedException: sleep interrupted
>>         at java.lang.Thread.sleep(Native Method)
>>         at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>> ZkController.java:905)
>>         at org.apache.solr.cloud.**ZkController.getLeaderProps(**
>> ZkController.java:875)
>>         at org.apache.solr.cloud.**ZkController.getLeader(**
>> ZkController.java:839)
>>         ... 25 more
>>
>> I have attached a minimal set of configuration files which are needed to
>> replicate this error, also containing the log files for the commands I have
>> run in the order above.
>>
>> I use the following versions of:
>> Solr: 4.2.1
>> ZooKeeper: 3.4.5
>> Tomcat 7.0.27
>>
>> app01: ZooKeeper
>> app02: ZooKeeper, Solr (in Tomcat)
>> app03: ZooKeeper, Solr (in Tomcat)
>>
>> The same procedure does work if I use Solr 4.0, but the BUG (
>> https://issues.apache.org/**jira/browse/SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
>> ) prevents me from using 4.0 in this scenario.
>>
>> If you need anything else, please just say so.
>>
>> Thanks for your help
>>
>> Kind Regards
>> Alexander
>>
>>
>
>
> --
> Joel Bernstein
> Professional Services LucidWorks
>



-- 
Joel Bernstein
Professional Services LucidWorks

Re: Empty Solr 4.2.1 can not create Collection

Posted by Joel Bernstein <jo...@gmail.com>.
The steps that I use to setup the collection are slightly different:


1) Start zk and upconfig the config set. Your approach is same.
2) Start appservers with Solr zkHost set to the zk started in step 1.
3) Use a core admin command to spin up a new core and collection.


http://app01/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&numShards=1&collection.configName=storage-conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
&shard=shard1

This will spin up the new collection and initial core. I'm not using a
replication factor because the following commands manually bind the
replicas.

4) Spin up replica with a core admin command:
http://app02/solr/admin/cores?action=CREATE&name=storage-core&collection=storage&<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
shard=shard1

5) Same command as above on the 3rd server to spin up another replica.

This will spin up a new core and bind it to shard1 of the storage
collection.





On Mon, Apr 8, 2013 at 9:34 AM, A.Eibner <a_...@yahoo.de> wrote:

> Hi,
>
> I have a problem with setting up my solr cloud environment (on three
> machines).
> If I want to create my collections from scratch I do the following:
>
> *) Start ZooKeeper on all machines.
>
> *) Upload the configuration (on app02) for the collection via the
> following command:
>     zkcli.sh -cmd upconfig --zkhost app01:4181,app02:4181,app03:**4181
> --confdir config/solr/storage/conf/ --confname storage-conf
>
> *) Linking the configuration (on app02) via the following command:
>     zkcli.sh -cmd linkconfig --collection storage --confname storage-conf
> --zkhost app01:4181,app02:4181,app03:**4181
>
> *) Start Tomcats (containing Solr) on app02,app03
>
> *) Create Collection via:
> http://app03/solr/admin/**collections?action=CREATE&**
> name=storage&numShards=1&**replicationFactor=2&**
> collection.configName=storage-**conf<http://app03/solr/admin/collections?action=CREATE&name=storage&numShards=1&replicationFactor=2&collection.configName=storage-conf>
>
> This creates the replication of the shard on app02 and app03, but neither
> of them is marked as leader, both are marked as DOWN.
> And after wards I can not access the collection.
> In the browser I get:
> "SEVERE: org.apache.solr.common.**SolrException: no servers hosting
> shard:"
>
> In the log files the following error is present:
> SEVERE: Error from shard: app02:9985/solr
> org.apache.solr.common.**SolrException: Error CREATEing SolrCore
> 'storage_shard1_replica1':
>         at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
> HttpSolrServer.java:404)
>         at org.apache.solr.client.solrj.**impl.HttpSolrServer.request(**
> HttpSolrServer.java:181)
>         at org.apache.solr.handler.**component.HttpShardHandler$1.**
> call(HttpShardHandler.java:**172)
>         at org.apache.solr.handler.**component.HttpShardHandler$1.**
> call(HttpShardHandler.java:**135)
>         at java.util.concurrent.**FutureTask$Sync.innerRun(**
> FutureTask.java:334)
>         at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>         at java.util.concurrent.**Executors$RunnableAdapter.**
> call(Executors.java:471)
>         at java.util.concurrent.**FutureTask$Sync.innerRun(**
> FutureTask.java:334)
>         at java.util.concurrent.**FutureTask.run(FutureTask.**java:166)
>         at java.util.concurrent.**ThreadPoolExecutor.runWorker(**
> ThreadPoolExecutor.java:1110)
>         at java.util.concurrent.**ThreadPoolExecutor$Worker.run(**
> ThreadPoolExecutor.java:603)
>         at java.lang.Thread.run(Thread.**java:722)
> Caused by: org.apache.solr.common.cloud.**ZooKeeperException:
>         at org.apache.solr.core.**CoreContainer.registerInZk(**
> CoreContainer.java:922)
>         at org.apache.solr.core.**CoreContainer.registerCore(**
> CoreContainer.java:892)
>         at org.apache.solr.core.**CoreContainer.register(**
> CoreContainer.java:841)
>         at org.apache.solr.handler.admin.**CoreAdminHandler.**
> handleCreateAction(**CoreAdminHandler.java:479)
>         ... 19 more
> Caused by: org.apache.solr.common.**SolrException: Error getting leader
> from zk for shard shard1
>         at org.apache.solr.cloud.**ZkController.getLeader(**
> ZkController.java:864)
>         at org.apache.solr.cloud.**ZkController.register(**
> ZkController.java:776)
>         at org.apache.solr.cloud.**ZkController.register(**
> ZkController.java:727)
>         at org.apache.solr.core.**CoreContainer.registerInZk(**
> CoreContainer.java:908)
>         ... 22 more
> Caused by: java.lang.**InterruptedException: sleep interrupted
>         at java.lang.Thread.sleep(Native Method)
>         at org.apache.solr.cloud.**ZkController.getLeaderProps(**
> ZkController.java:905)
>         at org.apache.solr.cloud.**ZkController.getLeaderProps(**
> ZkController.java:875)
>         at org.apache.solr.cloud.**ZkController.getLeader(**
> ZkController.java:839)
>         ... 25 more
>
> I have attached a minimal set of configuration files which are needed to
> replicate this error, also containing the log files for the commands I have
> run in the order above.
>
> I use the following versions of:
> Solr: 4.2.1
> ZooKeeper: 3.4.5
> Tomcat 7.0.27
>
> app01: ZooKeeper
> app02: ZooKeeper, Solr (in Tomcat)
> app03: ZooKeeper, Solr (in Tomcat)
>
> The same procedure does work if I use Solr 4.0, but the BUG (
> https://issues.apache.org/**jira/browse/SOLR-3939<https://issues.apache.org/jira/browse/SOLR-3939>
> ) prevents me from using 4.0 in this scenario.
>
> If you need anything else, please just say so.
>
> Thanks for your help
>
> Kind Regards
> Alexander
>
>


-- 
Joel Bernstein
Professional Services LucidWorks