mirror of https://github.com/apache/lucene.git
SOLR-13942: /api/cluster/zk/* to fetch raw ZK data
This commit is contained in:
parent
286d22717b
commit
bc6fa3b650
|
@ -65,6 +65,8 @@ New Features
|
|||
* SOLR-13965: In GraphHandler, support <expressible> configuration and deprecate <streamFunctions> configuration.
|
||||
(Eric Pugh via Christine Poerschke)
|
||||
|
||||
* SOLR-13942: /api/cluster/zk/* to fetch raw ZK data (noble)
|
||||
|
||||
Improvements
|
||||
---------------------
|
||||
* SOLR-14120: Define JavaScript methods 'includes' and 'startsWith' to ensure AdminUI can be displayed when using
|
||||
|
|
|
@ -96,6 +96,7 @@ import org.apache.solr.handler.admin.MetricsHistoryHandler;
|
|||
import org.apache.solr.handler.admin.SecurityConfHandler;
|
||||
import org.apache.solr.handler.admin.SecurityConfHandlerLocal;
|
||||
import org.apache.solr.handler.admin.SecurityConfHandlerZk;
|
||||
import org.apache.solr.handler.admin.ZkRead;
|
||||
import org.apache.solr.handler.admin.ZookeeperInfoHandler;
|
||||
import org.apache.solr.handler.admin.ZookeeperStatusHandler;
|
||||
import org.apache.solr.handler.component.ShardHandlerFactory;
|
||||
|
@ -674,6 +675,7 @@ public class CoreContainer {
|
|||
packageLoader = new PackageLoader(this);
|
||||
containerHandlers.getApiBag().register(new AnnotatedApi(packageLoader.getPackageAPI().editAPI), Collections.EMPTY_MAP);
|
||||
containerHandlers.getApiBag().register(new AnnotatedApi(packageLoader.getPackageAPI().readAPI), Collections.EMPTY_MAP);
|
||||
containerHandlers.getApiBag().register(new AnnotatedApi(new ZkRead(this)), Collections.EMPTY_MAP);
|
||||
}
|
||||
|
||||
MDCLoggingContext.setNode(this);
|
||||
|
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.solr.handler.admin;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.solr.api.Command;
|
||||
import org.apache.solr.api.EndPoint;
|
||||
import org.apache.solr.client.solrj.SolrRequest;
|
||||
import org.apache.solr.client.solrj.impl.BinaryResponseParser;
|
||||
import org.apache.solr.common.MapWriter;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.MapSolrParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ContentStreamBase;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
import org.apache.solr.core.CoreContainer;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import static org.apache.solr.common.params.CommonParams.OMIT_HEADER;
|
||||
import static org.apache.solr.common.params.CommonParams.WT;
|
||||
import static org.apache.solr.response.RawResponseWriter.CONTENT;
|
||||
import static org.apache.solr.security.PermissionNameProvider.Name.COLL_READ_PERM;
|
||||
|
||||
/**Exposes the content of the Zookeeper
|
||||
*
|
||||
*/
|
||||
@EndPoint(path = "/cluster/zk/*",
|
||||
method = SolrRequest.METHOD.GET,
|
||||
permission = COLL_READ_PERM)
|
||||
public class ZkRead {
|
||||
private final CoreContainer coreContainer;
|
||||
|
||||
public ZkRead(CoreContainer coreContainer) {
|
||||
this.coreContainer = coreContainer;
|
||||
}
|
||||
|
||||
@Command
|
||||
public void get(SolrQueryRequest req, SolrQueryResponse rsp) {
|
||||
String path = req.getPathTemplateValues().get("*");
|
||||
if (path == null || path.isEmpty()) path = "/";
|
||||
byte[] d = null;
|
||||
try {
|
||||
List<String> l = coreContainer.getZkController().getZkClient().getChildren(path, null, false);
|
||||
if (l != null && !l.isEmpty()) {
|
||||
String prefix = path.endsWith("/") ? path : path + "/";
|
||||
|
||||
rsp.add(path, (MapWriter) ew -> {
|
||||
for (String s : l) {
|
||||
try {
|
||||
Stat stat = coreContainer.getZkController().getZkClient().exists(prefix + s, null, false);
|
||||
ew.put(s, (MapWriter) ew1 -> {
|
||||
ew1.put("version", stat.getVersion());
|
||||
ew1.put("aversion", stat.getAversion());
|
||||
ew1.put("children", stat.getNumChildren());
|
||||
ew1.put("ctime", stat.getCtime());
|
||||
ew1.put("cversion", stat.getCversion());
|
||||
ew1.put("czxid", stat.getCzxid());
|
||||
ew1.put("ephemeralOwner", stat.getEphemeralOwner());
|
||||
ew1.put("mtime", stat.getMtime());
|
||||
ew1.put("mzxid", stat.getMzxid());
|
||||
ew1.put("pzxid", stat.getPzxid());
|
||||
ew1.put("dataLength", stat.getDataLength());
|
||||
});
|
||||
} catch (Exception e) {
|
||||
ew.put("s", Collections.singletonMap("error", e.getMessage()));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
} else {
|
||||
d = coreContainer.getZkController().getZkClient().getData(path, null, null, false);
|
||||
if (d == null || d.length == 0) {
|
||||
rsp.add(path, null);
|
||||
return;
|
||||
}
|
||||
|
||||
Map<String, String> map = new HashMap<>(1);
|
||||
map.put(WT, "raw");
|
||||
map.put(OMIT_HEADER, "true");
|
||||
req.setParams(SolrParams.wrapDefaults(new MapSolrParams(map), req.getParams()));
|
||||
|
||||
|
||||
rsp.add(CONTENT, new ContentStreamBase.ByteArrayStream(d, null,
|
||||
d[0] == '{' ? CommonParams.JSON_MIME : BinaryResponseParser.BINARY_CONTENT_TYPE));
|
||||
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
rsp.add(CONTENT, new ContentStreamBase.StringStream(Utils.toJSONString(Collections.singletonMap("error", e.getMessage()))));
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
}
|
||||
|
||||
}
|
|
@ -37,6 +37,7 @@ import org.apache.solr.cloud.SolrCloudTestCase;
|
|||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -74,6 +75,15 @@ public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
|
|||
super.tearDown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZkread() throws Exception {
|
||||
URL baseUrl = cluster.getJettySolrRunner(0).getBaseUrl();
|
||||
try( HttpSolrClient client = new HttpSolrClient.Builder(baseUrl.toString()).build()){
|
||||
Object o = Utils.executeGET(client.getHttpClient(), baseUrl.toString().replace("/solr", "/api"+ "/cluster/zk/security.json"), Utils.JSONCONSUMER );
|
||||
assertNotNull(o);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
Test the monitoring endpoint, used in the Cloud => ZkStatus Admin UI screen
|
||||
NOTE: We do not currently test with multiple zookeepers, but the only difference is that there are multiple "details" objects and mode is "ensemble"...
|
||||
|
@ -87,13 +97,13 @@ public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
|
|||
NamedList<Object> nl = solr.httpUriRequest(mntrReq).future.get(10000, TimeUnit.MILLISECONDS);
|
||||
|
||||
assertEquals("zkStatus", nl.getName(1));
|
||||
Map<String,Object> zkStatus = (Map<String,Object>) nl.get("zkStatus");
|
||||
Map<String, Object> zkStatus = (Map<String, Object>) nl.get("zkStatus");
|
||||
assertEquals("green", zkStatus.get("status"));
|
||||
assertEquals("standalone", zkStatus.get("mode"));
|
||||
assertEquals(1L, zkStatus.get("ensembleSize"));
|
||||
List<Object> detailsList = (List<Object>)zkStatus.get("details");
|
||||
List<Object> detailsList = (List<Object>) zkStatus.get("details");
|
||||
assertEquals(1, detailsList.size());
|
||||
Map<String,Object> details = (Map<String,Object>) detailsList.get(0);
|
||||
Map<String, Object> details = (Map<String, Object>) detailsList.get(0);
|
||||
assertEquals(true, details.get("ok"));
|
||||
assertTrue(Integer.parseInt((String) details.get("zk_znode_count")) > 50);
|
||||
solr.close();
|
||||
|
@ -106,12 +116,12 @@ public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
|
|||
when(zkStatusHandler.getZkRawResponse("zoo1:2181", "ruok")).thenReturn(Arrays.asList("imok"));
|
||||
when(zkStatusHandler.getZkRawResponse("zoo1:2181", "mntr")).thenReturn(
|
||||
Arrays.asList("zk_version\t3.5.5-390fe37ea45dee01bf87dc1c042b5e3dcce88653, built on 05/03/2019 12:07 GMT",
|
||||
"zk_avg_latency\t1"));
|
||||
"zk_avg_latency\t1"));
|
||||
when(zkStatusHandler.getZkRawResponse("zoo1:2181", "conf")).thenReturn(
|
||||
Arrays.asList("clientPort=2181",
|
||||
"secureClientPort=-1",
|
||||
"thisIsUnexpected",
|
||||
"membership: "));
|
||||
"secureClientPort=-1",
|
||||
"thisIsUnexpected",
|
||||
"membership: "));
|
||||
|
||||
when(zkStatusHandler.getZkRawResponse("zoo2:2181", "ruok")).thenReturn(Arrays.asList(""));
|
||||
|
||||
|
|
Loading…
Reference in New Issue