Skip to content

Commit ce5c733

Browse files
authored
SOLR-16116: Remove retryOnConnLoss parameter (#3969)
This removes parameter retryOnConnLoss everywhere from the codebase. This parameter was already fully ignored since the move to Curator.
1 parent 2fc349c commit ce5c733

File tree

128 files changed

+636
-828
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

128 files changed

+636
-828
lines changed
Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,8 @@
1+
title: Remove parameter retryOnConnLoss everywhere from the codebase. This parameter
2+
was already fully ignored since the move to Curator.
3+
type: removed
4+
authors:
5+
- name: Pierre Salagnac
6+
links:
7+
- name: SOLR-16116
8+
url: https://issues.apache.org/jira/browse/SOLR-16116

solr/core/src/java/org/apache/solr/api/ZkClusterPluginsSource.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ public Map<String, Object> plugins() throws IOException {
7171
try {
7272
Map<String, Object> clusterPropsJson =
7373
(Map<String, Object>)
74-
Utils.fromJSON(zkClient.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat(), true));
74+
Utils.fromJSON(zkClient.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat()));
7575
return Map.copyOf(
7676
(Map<String, Object>)
7777
clusterPropsJson.computeIfAbsent(

solr/core/src/java/org/apache/solr/cli/AuthTool.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -249,7 +249,7 @@ private void handleBasicAuth(CommandLine cli) throws Exception {
249249
if (!updateIncludeFileOnly) {
250250
echoIfVerbose("Uploading following security.json: " + securityJson);
251251
try (SolrZkClient zkClient = CLIUtils.getSolrZkClient(cli, zkHost)) {
252-
zkClient.setData("/security.json", securityJson.getBytes(StandardCharsets.UTF_8), true);
252+
zkClient.setData("/security.json", securityJson.getBytes(StandardCharsets.UTF_8));
253253
}
254254
}
255255

@@ -309,8 +309,8 @@ private void handleBasicAuth(CommandLine cli) throws Exception {
309309

310310
private void checkSecurityJsonExists(SolrZkClient zkClient)
311311
throws KeeperException, InterruptedException {
312-
if (zkClient.exists("/security.json", true)) {
313-
byte[] oldSecurityBytes = zkClient.getData("/security.json", null, null, true);
312+
if (zkClient.exists("/security.json")) {
313+
byte[] oldSecurityBytes = zkClient.getData("/security.json", null, null);
314314
if (!"{}".equals(new String(oldSecurityBytes, StandardCharsets.UTF_8).trim())) {
315315
CLIO.out(
316316
"Security is already enabled. You can disable it with 'bin/solr auth disable'. Existing security.json: \n"
@@ -332,7 +332,7 @@ private void clearSecurityJson(CommandLine cli, boolean updateIncludeFileOnly) t
332332
echoIfVerbose("Uploading following security.json: {}");
333333

334334
try (SolrZkClient zkClient = CLIUtils.getSolrZkClient(cli, zkHost)) {
335-
zkClient.setData("/security.json", "{}".getBytes(StandardCharsets.UTF_8), true);
335+
zkClient.setData("/security.json", "{}".getBytes(StandardCharsets.UTF_8));
336336
}
337337
}
338338
}

solr/core/src/java/org/apache/solr/cli/CreateTool.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -248,9 +248,7 @@ protected void createCollection(CloudSolrClient cloudSolrClient, CommandLine cli
248248
boolean configExistsInZk =
249249
confName != null
250250
&& !confName.trim().isEmpty()
251-
&& ZkStateReader.from(cloudSolrClient)
252-
.getZkClient()
253-
.exists("/configs/" + confName, true);
251+
&& ZkStateReader.from(cloudSolrClient).getZkClient().exists("/configs/" + confName);
254252

255253
if (configExistsInZk) {
256254
echo("Re-using existing configuration directory " + confName);

solr/core/src/java/org/apache/solr/cli/ZkMkrootTool.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -79,7 +79,7 @@ public void runImpl(CommandLine cli) throws Exception {
7979
echoIfVerbose("\nConnecting to ZooKeeper at " + zkHost + " ...");
8080

8181
echo("Creating ZooKeeper path " + znode + " on ZooKeeper at " + zkHost);
82-
zkClient.makePath(znode, failOnExists, true);
82+
zkClient.makePath(znode, failOnExists);
8383
} catch (Exception e) {
8484
log.error("Could not complete mkroot operation for reason: ", e);
8585
throw (e);

solr/core/src/java/org/apache/solr/cli/ZkRmTool.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,7 @@ public void runImpl(CommandLine cli) throws Exception {
6767
}
6868
echoIfVerbose("\nConnecting to ZooKeeper at " + zkHost + " ...");
6969
try (SolrZkClient zkClient = CLIUtils.getSolrZkClient(cli, zkHost)) {
70-
if (!recursive && zkClient.getChildren(znode, null, true).size() != 0) {
70+
if (!recursive && zkClient.getChildren(znode, null).size() != 0) {
7171
throw new SolrServerException(
7272
"ZooKeeper node " + znode + " has children and recursive has NOT been specified.");
7373
}

solr/core/src/java/org/apache/solr/cloud/CloudUtil.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -129,10 +129,10 @@ public static String unifiedResourcePath(SolrResourceLoader loader) {
129129
public static Map<String, byte[]> getTrustedKeys(SolrZkClient zk, String dir) {
130130
Map<String, byte[]> result = new HashMap<>();
131131
try {
132-
List<String> children = zk.getChildren("/keys/" + dir, null, true);
132+
List<String> children = zk.getChildren("/keys/" + dir, null);
133133
for (String key : children) {
134134
if (key.endsWith(".der"))
135-
result.put(key, zk.getData("/keys/" + dir + "/" + key, null, null, true));
135+
result.put(key, zk.getData("/keys/" + dir + "/" + key, null, null));
136136
}
137137
} catch (KeeperException.NoNodeException e) {
138138
log.info("Error fetching key names");

solr/core/src/java/org/apache/solr/cloud/DistributedApiAsyncTracker.java

Lines changed: 7 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -323,8 +323,8 @@ enum State {
323323
this.rootNodePath = rootNodePath;
324324

325325
try {
326-
if (!zkClient.exists(rootNodePath, true)) {
327-
zkClient.makePath(rootNodePath, new byte[0], CreateMode.PERSISTENT, true);
326+
if (!zkClient.exists(rootNodePath)) {
327+
zkClient.makePath(rootNodePath, new byte[0], CreateMode.PERSISTENT);
328328
}
329329
} catch (KeeperException.NodeExistsException nee) {
330330
// Some other thread (on this or another JVM) beat us to create the node, that's ok, the
@@ -341,27 +341,25 @@ void createNewInFlightTask(String asyncId) throws KeeperException, InterruptedEx
341341
zkClient.create(
342342
getPath(asyncId),
343343
State.SUBMITTED.shorthand.getBytes(StandardCharsets.UTF_8),
344-
CreateMode.EPHEMERAL,
345-
true);
344+
CreateMode.EPHEMERAL);
346345
}
347346

348347
void setTaskRunning(String asyncId) throws KeeperException, InterruptedException {
349-
zkClient.setData(
350-
getPath(asyncId), State.RUNNING.shorthand.getBytes(StandardCharsets.UTF_8), true);
348+
zkClient.setData(getPath(asyncId), State.RUNNING.shorthand.getBytes(StandardCharsets.UTF_8));
351349
}
352350

353351
void deleteInFlightTask(String asyncId) throws KeeperException, InterruptedException {
354-
zkClient.delete(getPath(asyncId), -1, true);
352+
zkClient.delete(getPath(asyncId), -1);
355353
}
356354

357355
State getInFlightState(String asyncId) throws KeeperException, InterruptedException {
358-
if (!zkClient.exists(getPath(asyncId), true)) {
356+
if (!zkClient.exists(getPath(asyncId))) {
359357
return State.NOT_FOUND;
360358
}
361359

362360
final byte[] bytes;
363361
try {
364-
bytes = zkClient.getData(getPath(asyncId), null, null, true);
362+
bytes = zkClient.getData(getPath(asyncId), null, null);
365363
} catch (KeeperException.NoNodeException nne) {
366364
// Unlikely race, but not impossible...
367365
if (log.isInfoEnabled()) {

solr/core/src/java/org/apache/solr/cloud/DistributedClusterStateUpdater.java

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -552,7 +552,7 @@ private void doStateDotJsonCasUpdate(ClusterState updatedState)
552552
if (updater.isCollectionCreation()) {
553553
// The state.json file does not exist yet (more precisely it is assumed not to exist)
554554
log.debug("going to create collection {}", jsonPath);
555-
zkStateReader.getZkClient().create(jsonPath, stateJson, CreateMode.PERSISTENT, true);
555+
zkStateReader.getZkClient().create(jsonPath, stateJson, CreateMode.PERSISTENT);
556556
} else {
557557
// We're updating an existing state.json
558558
if (log.isDebugEnabled()) {
@@ -561,9 +561,7 @@ private void doStateDotJsonCasUpdate(ClusterState updatedState)
561561
jsonPath,
562562
collection.getZNodeVersion());
563563
}
564-
zkStateReader
565-
.getZkClient()
566-
.setData(jsonPath, stateJson, collection.getZNodeVersion(), true);
564+
zkStateReader.getZkClient().setData(jsonPath, stateJson, collection.getZNodeVersion());
567565
}
568566
}
569567
}
@@ -577,7 +575,7 @@ private void doStateDotJsonCasUpdate(ClusterState updatedState)
577575
private ClusterState fetchStateForCollection() throws KeeperException, InterruptedException {
578576
String collectionStatePath = DocCollection.getCollectionPath(updater.getCollectionName());
579577
Stat stat = new Stat();
580-
byte[] data = zkStateReader.getZkClient().getData(collectionStatePath, null, stat, true);
578+
byte[] data = zkStateReader.getZkClient().getData(collectionStatePath, null, stat);
581579

582580
// This factory method can detect a missing configName and supply it by reading it from the
583581
// old ZK location.
@@ -890,7 +888,7 @@ public static void executeNodeDownStateUpdate(String nodeName, ZkStateReader zkS
890888

891889
try {
892890
final List<String> collectionNames =
893-
zkStateReader.getZkClient().getChildren(COLLECTIONS_ZKNODE, null, true);
891+
zkStateReader.getZkClient().getChildren(COLLECTIONS_ZKNODE, null);
894892

895893
// Collections are totally independent of each other. Multiple threads could share the load
896894
// here (need a ZK connection for each though).

solr/core/src/java/org/apache/solr/cloud/DistributedMap.java

Lines changed: 9 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -68,8 +68,7 @@ private void assertKeyFormat(String trackingId) {
6868

6969
public void put(String trackingId, byte[] data) throws KeeperException, InterruptedException {
7070
assertKeyFormat(trackingId);
71-
zookeeper.makePath(
72-
dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, false, true);
71+
zookeeper.makePath(dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, false);
7372
}
7473

7574
/**
@@ -81,25 +80,24 @@ public boolean putIfAbsent(String trackingId, byte[] data)
8180
throws KeeperException, InterruptedException {
8281
assertKeyFormat(trackingId);
8382
try {
84-
zookeeper.makePath(
85-
dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, true, true);
83+
zookeeper.makePath(dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, true);
8684
return true;
8785
} catch (NodeExistsException e) {
8886
return false;
8987
}
9088
}
9189

9290
public byte[] get(String trackingId) throws KeeperException, InterruptedException {
93-
return zookeeper.getData(dir + "/" + PREFIX + trackingId, null, null, true);
91+
return zookeeper.getData(dir + "/" + PREFIX + trackingId, null, null);
9492
}
9593

9694
public boolean contains(String trackingId) throws KeeperException, InterruptedException {
97-
return zookeeper.exists(dir + "/" + PREFIX + trackingId, true);
95+
return zookeeper.exists(dir + "/" + PREFIX + trackingId);
9896
}
9997

10098
public int size() throws KeeperException, InterruptedException {
10199
Stat stat = new Stat();
102-
zookeeper.getData(dir, null, stat, true);
100+
zookeeper.getData(dir, null, stat);
103101
return stat.getNumChildren();
104102
}
105103

@@ -110,7 +108,7 @@ public int size() throws KeeperException, InterruptedException {
110108
public boolean remove(String trackingId) throws KeeperException, InterruptedException {
111109
final var path = dir + "/" + PREFIX + trackingId;
112110
try {
113-
zookeeper.delete(path, -1, true);
111+
zookeeper.delete(path, -1);
114112
} catch (KeeperException.NoNodeException e) {
115113
return false;
116114
} catch (KeeperException.NotEmptyException hack) {
@@ -123,15 +121,15 @@ public boolean remove(String trackingId) throws KeeperException, InterruptedExce
123121

124122
/** Helper method to clear all child nodes for a parent node. */
125123
public void clear() throws KeeperException, InterruptedException {
126-
List<String> childNames = zookeeper.getChildren(dir, null, true);
124+
List<String> childNames = zookeeper.getChildren(dir, null);
127125
for (String childName : childNames) {
128-
zookeeper.delete(dir + "/" + childName, -1, true);
126+
zookeeper.delete(dir + "/" + childName, -1);
129127
}
130128
}
131129

132130
/** Returns the keys of all the elements in the map */
133131
public Collection<String> keys() throws KeeperException, InterruptedException {
134-
List<String> childs = zookeeper.getChildren(dir, null, true);
132+
List<String> childs = zookeeper.getChildren(dir, null);
135133
final List<String> ids = new ArrayList<>(childs.size());
136134
childs.stream().forEach((child) -> ids.add(child.substring(PREFIX.length())));
137135
return ids;

0 commit comments

Comments
 (0)