You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by no...@apache.org on 2023/03/30 00:07:02 UTC

[solr] branch main updated: ZkStateReader : refactor out `security.json` handling (#1497)

This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/main by this push:
     new b45c7c7c855 ZkStateReader : refactor out `security.json` handling (#1497)
b45c7c7c855 is described below

commit b45c7c7c855cf2624606823001b23e88968a1af0
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Thu Mar 30 11:06:55 2023 +1100

    ZkStateReader : refactor out `security.json` handling (#1497)
---
 .../solr/common/cloud/SecurityNodeWatcher.java     | 129 +++++++++++++++++++++
 .../apache/solr/common/cloud/ZkStateReader.java    |  97 ++--------------
 2 files changed, 136 insertions(+), 90 deletions(-)

diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SecurityNodeWatcher.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SecurityNodeWatcher.java
new file mode 100644
index 00000000000..487fa6b427b
--- /dev/null
+++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SecurityNodeWatcher.java
@@ -0,0 +1,129 @@
+/*
+ * 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.common.cloud;
+
+import static java.util.Collections.emptyMap;
+
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import org.apache.solr.common.Callable;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SecurityNodeWatcher implements Watcher {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final ZkStateReader zkStateReader;
+  private ZkStateReader.ConfigData securityData;
+
+  private final Callable<SolrZkClient.NodeData> callback;
+
+  @SuppressWarnings("unchecked")
+  public SecurityNodeWatcher(ZkStateReader zkStateReader, Runnable securityNodeListener) {
+    this.zkStateReader = zkStateReader;
+    callback =
+        data -> {
+          ZkStateReader.ConfigData cd = new ZkStateReader.ConfigData();
+          cd.data =
+              data.data == null || data.data.length == 0
+                  ? emptyMap()
+                  : Utils.getDeepCopy((Map) Utils.fromJSON(data.data), 4, false);
+          cd.version = data.stat == null ? -1 : data.stat.getVersion();
+          securityData = cd;
+          if (securityNodeListener != null) securityNodeListener.run();
+        };
+  }
+
+  @Override
+  public void process(WatchedEvent event) {
+    // session events are not change events, and do not remove the watcher
+    if (Event.EventType.None.equals(event.getType())) {
+      return;
+    }
+    try {
+      synchronized (this) {
+        log.debug("Updating [{}] ... ", ZkStateReader.SOLR_SECURITY_CONF_PATH);
+
+        // remake watch
+
+        SolrZkClient.NodeData data =
+            new SolrZkClient.NodeData(new Stat(), "{}".getBytes(StandardCharsets.UTF_8));
+        if (Event.EventType.NodeDeleted.equals(event.getType())) {
+          // Node deleted, just recreate watch without attempting a read - SOLR-9679
+          zkStateReader.getZkClient().exists(ZkStateReader.SOLR_SECURITY_CONF_PATH, this, true);
+        } else {
+          data =
+              zkStateReader
+                  .getZkClient()
+                  .getNode(ZkStateReader.SOLR_SECURITY_CONF_PATH, this, true);
+        }
+        try {
+          callback.call(data);
+        } catch (Exception e) {
+          log.error("Error running collections node listener", e);
+        }
+      }
+    } catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
+      log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: ", e);
+    } catch (KeeperException e) {
+      log.error("A ZK error has occurred", e);
+      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
+    } catch (InterruptedException e) {
+      // Restore the interrupted status
+      Thread.currentThread().interrupt();
+      log.warn("Interrupted", e);
+    }
+  }
+
+  void register() throws InterruptedException, KeeperException {
+    zkStateReader.getZkClient().exists(ZkStateReader.SOLR_SECURITY_CONF_PATH, this, true);
+    securityData = getSecurityProps(true);
+  }
+
+  @SuppressWarnings("unchecked")
+  ZkStateReader.ConfigData getSecurityProps(boolean getFresh) {
+    if (!getFresh) {
+      if (securityData == null) return new ZkStateReader.ConfigData(emptyMap(), -1);
+      return new ZkStateReader.ConfigData(securityData.data, securityData.version);
+    }
+    try {
+      if (zkStateReader.getZkClient().exists(ZkStateReader.SOLR_SECURITY_CONF_PATH, true)) {
+        SolrZkClient.NodeData d =
+            zkStateReader.getZkClient().getNode(ZkStateReader.SOLR_SECURITY_CONF_PATH, null, true);
+        return d != null && d.data.length > 0
+            ? new ZkStateReader.ConfigData(
+                (Map<String, Object>) Utils.fromJSON(d.data), d.stat.getVersion())
+            : null;
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR, "Error reading security properties", e);
+    } catch (KeeperException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR, "Error reading security properties", e);
+    }
+    return null;
+  }
+}
diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 259141d0f54..3d10656717c 100644
--- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -20,7 +20,6 @@ import static java.util.Collections.emptyMap;
 import static java.util.Collections.emptySortedSet;
 
 import java.lang.invoke.MethodHandles;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -50,7 +49,6 @@ import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
 import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.Callable;
 import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -59,7 +57,6 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.CommonTestInjection;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.ObjectReleaseTracker;
-import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.KeeperException;
@@ -189,10 +186,6 @@ public class ZkStateReader implements SolrCloseable {
 
   private volatile Map<String, Object> clusterProperties = Collections.emptyMap();
 
-  private ConfigData securityData;
-
-  private final Runnable securityNodeListener;
-
   /**
    * Collections with active watches. The {@link StatefulCollectionWatch} inside for each collection
    * might also contain the latest DocCollection (state) observed
@@ -399,6 +392,7 @@ public class ZkStateReader implements SolrCloseable {
   private volatile boolean closed = false;
 
   private Set<CountDownLatch> waitLatches = ConcurrentHashMap.newKeySet();
+  private final SecurityNodeWatcher securityNodeWatcher;
 
   public ZkStateReader(SolrZkClient zkClient) {
     this(zkClient, null);
@@ -407,7 +401,7 @@ public class ZkStateReader implements SolrCloseable {
   public ZkStateReader(SolrZkClient zkClient, Runnable securityNodeListener) {
     this.zkClient = zkClient;
     this.closeClient = false;
-    this.securityNodeListener = securityNodeListener;
+    this.securityNodeWatcher = new SecurityNodeWatcher(this, securityNodeListener);
     assert ObjectReleaseTracker.track(this);
   }
 
@@ -435,7 +429,7 @@ public class ZkStateReader implements SolrCloseable {
                 })
             .build();
     this.closeClient = true;
-    this.securityNodeListener = null;
+    this.securityNodeWatcher = null;
 
     assert ObjectReleaseTracker.track(this);
   }
@@ -571,19 +565,8 @@ public class ZkStateReader implements SolrCloseable {
       refreshCollectionList(new CollectionsChildWatcher());
       refreshAliases(aliasesManager);
 
-      if (securityNodeListener != null) {
-        addSecurityNodeWatcher(
-            pair -> {
-              ConfigData cd = new ConfigData();
-              cd.data =
-                  pair.first() == null || pair.first().length == 0
-                      ? emptyMap()
-                      : Utils.getDeepCopy((Map) Utils.fromJSON(pair.first()), 4, false);
-              cd.version = pair.second() == null ? -1 : pair.second().getVersion();
-              securityData = cd;
-              securityNodeListener.run();
-            });
-        securityData = getSecurityProps(true);
+      if (securityNodeWatcher != null) {
+        securityNodeWatcher.register();
       }
 
       collectionPropsObservers.forEach(
@@ -602,53 +585,6 @@ public class ZkStateReader implements SolrCloseable {
     }
   }
 
-  private void addSecurityNodeWatcher(final Callable<Pair<byte[], Stat>> callback)
-      throws KeeperException, InterruptedException {
-    zkClient.exists(
-        SOLR_SECURITY_CONF_PATH,
-        new Watcher() {
-
-          @Override
-          public void process(WatchedEvent event) {
-            // session events are not change events, and do not remove the watcher
-            if (EventType.None.equals(event.getType())) {
-              return;
-            }
-            try {
-              synchronized (ZkStateReader.this.getUpdateLock()) {
-                log.debug("Updating [{}] ... ", SOLR_SECURITY_CONF_PATH);
-
-                // remake watch
-                final Stat stat = new Stat();
-                byte[] data = "{}".getBytes(StandardCharsets.UTF_8);
-                if (EventType.NodeDeleted.equals(event.getType())) {
-                  // Node deleted, just recreate watch without attempting a read - SOLR-9679
-                  getZkClient().exists(SOLR_SECURITY_CONF_PATH, this, true);
-                } else {
-                  data = getZkClient().getData(SOLR_SECURITY_CONF_PATH, this, stat, true);
-                }
-                try {
-                  callback.call(new Pair<>(data, stat));
-                } catch (Exception e) {
-                  log.error("Error running collections node listener", e);
-                }
-              }
-            } catch (KeeperException.ConnectionLossException
-                | KeeperException.SessionExpiredException e) {
-              log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: ", e);
-            } catch (KeeperException e) {
-              log.error("A ZK error has occurred", e);
-              throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "", e);
-            } catch (InterruptedException e) {
-              // Restore the interrupted status
-              Thread.currentThread().interrupt();
-              log.warn("Interrupted", e);
-            }
-          }
-        },
-        true);
-  }
-
   /**
    * Construct the total state view from all sources. Must hold {@link #getUpdateLock()} before
    * calling this.
@@ -1363,28 +1299,9 @@ public class ZkStateReader implements SolrCloseable {
    * Returns the content of /security.json from ZooKeeper as a Map If the files doesn't exist, it
    * returns null.
    */
-  @SuppressWarnings("unchecked")
   public ConfigData getSecurityProps(boolean getFresh) {
-    if (!getFresh) {
-      if (securityData == null) return new ConfigData(emptyMap(), -1);
-      return new ConfigData(securityData.data, securityData.version);
-    }
-    try {
-      Stat stat = new Stat();
-      if (getZkClient().exists(SOLR_SECURITY_CONF_PATH, true)) {
-        final byte[] data =
-            getZkClient().getData(ZkStateReader.SOLR_SECURITY_CONF_PATH, null, stat, true);
-        return data != null && data.length > 0
-            ? new ConfigData((Map<String, Object>) Utils.fromJSON(data), stat.getVersion())
-            : null;
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Error reading security properties", e);
-    } catch (KeeperException e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Error reading security properties", e);
-    }
-    return null;
+    if (securityNodeWatcher == null) return new ConfigData(emptyMap(), -1);
+    return securityNodeWatcher.getSecurityProps(getFresh);
   }
 
   /**