forked from mdodsworth/lucene-solr
/
CloudSolrServer.java
307 lines (269 loc) · 10.9 KB
/
CloudSolrServer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
package org.apache.solr.client.solrj.impl;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.net.MalformedURLException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.TimeoutException;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.request.IsUpdateRequest;
import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.zookeeper.KeeperException;
/**
* SolrJ client class to communicate with SolrCloud.
* Instances of this class communicate with Zookeeper to discover
* Solr endpoints for SolrCloud collections, and then use the
* {@link LBHttpSolrServer} to issue requests.
*/
public class CloudSolrServer extends SolrServer {
private volatile ZkStateReader zkStateReader;
private String zkHost; // the zk server address
private int zkConnectTimeout = 10000;
private int zkClientTimeout = 10000;
private volatile String defaultCollection;
private LBHttpSolrServer lbServer;
private HttpClient myClient;
Random rand = new Random();
private Object cachLock = new Object();
// since the state shouldn't change often, should be very cheap reads
private Map<String,List<String>> urlLists = new HashMap<String,List<String>>();
private Map<String,List<String>> leaderUrlLists = new HashMap<String,List<String>>();
private Map<String,List<String>> replicasLists = new HashMap<String,List<String>>();
private volatile int lastClusterStateHashCode;
private final boolean updatesToLeaders;
/**
* @param zkHost The client endpoint of the zookeeper quorum containing the cloud state,
* in the form HOST:PORT.
*/
public CloudSolrServer(String zkHost) throws MalformedURLException {
this.zkHost = zkHost;
this.myClient = HttpClientUtil.createClient(null);
this.lbServer = new LBHttpSolrServer(myClient);
this.updatesToLeaders = true;
}
/**
* @param zkHost The client endpoint of the zookeeper quorum containing the cloud state,
* in the form HOST:PORT.
* @param lbServer LBHttpSolrServer instance for requests.
*/
public CloudSolrServer(String zkHost, LBHttpSolrServer lbServer) {
this.zkHost = zkHost;
this.lbServer = lbServer;
this.updatesToLeaders = true;
}
/**
* @param zkHost The client endpoint of the zookeeper quorum containing the cloud state,
* in the form HOST:PORT.
* @param lbServer LBHttpSolrServer instance for requests.
* @param updatesToLeaders sends updates only to leaders - defaults to true
*/
public CloudSolrServer(String zkHost, LBHttpSolrServer lbServer, boolean updatesToLeaders) {
this.zkHost = zkHost;
this.lbServer = lbServer;
this.updatesToLeaders = updatesToLeaders;
}
public ZkStateReader getZkStateReader() {
return zkStateReader;
}
/** Sets the default collection for request */
public void setDefaultCollection(String collection) {
this.defaultCollection = collection;
}
/** Set the connect timeout to the zookeeper ensemble in ms */
public void setZkConnectTimeout(int zkConnectTimeout) {
this.zkConnectTimeout = zkConnectTimeout;
}
/** Set the timeout to the zookeeper ensemble in ms */
public void setZkClientTimeout(int zkClientTimeout) {
this.zkClientTimeout = zkClientTimeout;
}
/**
* Connect to the zookeeper ensemble.
* This is an optional method that may be used to force a connect before any other requests are sent.
*
*/
public void connect() {
if (zkStateReader == null) {
synchronized (this) {
if (zkStateReader == null) {
try {
ZkStateReader zk = new ZkStateReader(zkHost, zkConnectTimeout,
zkClientTimeout);
zk.createClusterStateWatchersAndUpdate();
zkStateReader = zk;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (KeeperException e) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (TimeoutException e) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
}
}
}
@Override
public NamedList<Object> request(SolrRequest request) throws SolrServerException, IOException {
connect();
// TODO: if you can hash here, you could favor the shard leader
ClusterState clusterState = zkStateReader.getClusterState();
boolean sendToLeaders = false;
List<String> replicas = null;
if (request instanceof IsUpdateRequest && updatesToLeaders) {
sendToLeaders = true;
replicas = new ArrayList<String>();
}
SolrParams reqParams = request.getParams();
if (reqParams == null) {
reqParams = new ModifiableSolrParams();
}
String collection = reqParams.get("collection", defaultCollection);
if (collection == null) {
throw new SolrServerException("No collection param specified on request and no default collection has been set.");
}
// Extract each comma separated collection name and store in a List.
List<String> collectionList = StrUtils.splitSmart(collection, ",", true);
// TODO: not a big deal because of the caching, but we could avoid looking at every shard
// when getting leaders if we tweaked some things
// Retrieve slices from the cloud state and, for each collection specified,
// add it to the Map of slices.
Map<String,Slice> slices = new HashMap<String,Slice>();
for (int i = 0; i < collectionList.size(); i++) {
String coll= collectionList.get(i);
ClientUtils.appendMap(coll, slices, clusterState.getSlices(coll));
}
Set<String> liveNodes = clusterState.getLiveNodes();
List<String> theUrlList;
synchronized (cachLock) {
System.out.println("work with collection:" + collection);
List<String> leaderUrlList = leaderUrlLists.get(collection);
List<String> urlList = urlLists.get(collection);
List<String> replicasList = replicasLists.get(collection);
if ((sendToLeaders && leaderUrlList == null) || (!sendToLeaders
&& urlList == null)
|| clusterState.hashCode() != this.lastClusterStateHashCode) {
// build a map of unique nodes
// TODO: allow filtering by group, role, etc
Map<String,ZkNodeProps> nodes = new HashMap<String,ZkNodeProps>();
List<String> urlList2 = new ArrayList<String>();
for (Slice slice : slices.values()) {
for (ZkNodeProps nodeProps : slice.getReplicasMap().values()) {
ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(nodeProps);
String node = coreNodeProps.getNodeName();
if (!liveNodes.contains(coreNodeProps.getNodeName())
|| !coreNodeProps.getState().equals(ZkStateReader.ACTIVE)) continue;
if (nodes.put(node, nodeProps) == null) {
if (!sendToLeaders || (sendToLeaders && coreNodeProps.isLeader())) {
String url = coreNodeProps.getCoreUrl();
urlList2.add(url);
} else if (sendToLeaders) {
String url = coreNodeProps.getCoreUrl();
replicas.add(url);
}
}
}
}
if (sendToLeaders) {
this.leaderUrlLists.put(collection, urlList2);
leaderUrlList = urlList2;
this.replicasLists.put(collection, replicas);
replicasList = replicas;
} else {
this.urlLists.put(collection, urlList2);
urlList = urlList2;
}
this.lastClusterStateHashCode = clusterState.hashCode();
}
if (sendToLeaders) {
theUrlList = new ArrayList<String>(leaderUrlList.size());
theUrlList.addAll(leaderUrlList);
} else {
theUrlList = new ArrayList<String>(urlList.size());
theUrlList.addAll(urlList);
}
Collections.shuffle(theUrlList, rand);
if (sendToLeaders) {
ArrayList<String> theReplicas = new ArrayList<String>(
replicasList.size());
theReplicas.addAll(replicasList);
Collections.shuffle(theReplicas, rand);
// System.out.println("leaders:" + theUrlList);
// System.out.println("replicas:" + theReplicas);
theUrlList.addAll(theReplicas);
}
}
// System.out.println("########################## MAKING REQUEST TO " + theUrlList);
LBHttpSolrServer.Req req = new LBHttpSolrServer.Req(request, theUrlList);
LBHttpSolrServer.Rsp rsp = lbServer.request(req);
return rsp.getResponse();
}
@Override
public void shutdown() {
if (zkStateReader != null) {
synchronized(this) {
if (zkStateReader!= null)
zkStateReader.close();
zkStateReader = null;
}
}
if (myClient!=null) {
myClient.getConnectionManager().shutdown();
}
}
public LBHttpSolrServer getLbServer() {
return lbServer;
}
// for tests
Map<String,List<String>> getUrlLists() {
return urlLists;
}
//for tests
Map<String,List<String>> getLeaderUrlLists() {
return leaderUrlLists;
}
//for tests
Map<String,List<String>> getReplicasLists() {
return replicasLists;
}
}