diff --git a/gradle/testing/randomization/policies/solr-tests.policy b/gradle/testing/randomization/policies/solr-tests.policy index 8183cf1e1e7..61df0871a35 100644 --- a/gradle/testing/randomization/policies/solr-tests.policy +++ b/gradle/testing/randomization/policies/solr-tests.policy @@ -50,6 +50,8 @@ grant { permission java.net.SocketPermission "127.0.0.1:4", "connect,resolve"; permission java.net.SocketPermission "127.0.0.1:6", "connect,resolve"; permission java.net.SocketPermission "127.0.0.1:8", "connect,resolve"; + // Used as an invalid ZK host + permission java.net.SocketPermission "----------:33332", "connect,resolve"; // Basic permissions needed for Lucene to work: permission java.util.PropertyPermission "*", "read,write"; diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index e61f29f122a..83327d8605b 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -23,6 +23,9 @@ Improvements * SOLR-17077: When a shard rejoins leader election, leave previous election only once to save unneeded calls to Zookeeper. (Pierre Salagnac) +* SOLR-16116: Apache Curator is now used to manage all Solr Zookeeper interactions. This should provide more stability in the Solr-Zookeeper interactions. + The solrj-zookeeper module, now has a dependency on curator. (Houston Putman, Kevin Risden, Mike Drob, David Smiley) + Optimizations --------------------- (No changes) diff --git a/solr/core/build.gradle b/solr/core/build.gradle index b36843fa568..92bbfbe1120 100644 --- a/solr/core/build.gradle +++ b/solr/core/build.gradle @@ -49,7 +49,6 @@ dependencies { api project(':solr:solrj-zookeeper') api project(':solr:solrj-streaming') - api 'io.dropwizard.metrics:metrics-core' implementation ('io.dropwizard.metrics:metrics-graphite', { exclude group: "com.rabbitmq", module: "amqp-client" @@ -125,6 +124,16 @@ dependencies { implementation 'org.eclipse.jetty.toolchain:jetty-servlet-api' // ZooKeeper + + implementation('org.apache.curator:curator-framework', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) + implementation('org.apache.curator:curator-client', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) + testImplementation('org.apache.curator:curator-test', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) implementation('org.apache.zookeeper:zookeeper', { exclude group: "org.apache.yetus", module: "audience-annotations" }) diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java index 544437c5d67..f058aefe295 100644 --- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java +++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java @@ -25,7 +25,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.solr.cloud.ZkController.ContextKey; -import org.apache.solr.common.AlreadyClosedException; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkMaintenanceUtils; @@ -107,7 +106,7 @@ private void checkIfIamLeader(final ElectionContext context, boolean replacement // If any double-registrations exist for me, remove all but this latest one! // TODO: can we even get into this state? - String prefix = zkClient.getZooKeeper().getSessionId() + "-" + context.id + "-"; + String prefix = zkClient.getZkSessionId() + "-" + context.id + "-"; Iterator it = seqs.iterator(); while (it.hasNext()) { String elec = it.next(); @@ -227,7 +226,7 @@ public int joinElection(ElectionContext context, boolean replacement, boolean jo final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE; - long sessionId = zkClient.getZooKeeper().getSessionId(); + long sessionId = zkClient.getZkSessionId(); String id = sessionId + "-" + context.id; String leaderSeqPath = null; boolean cont = true; @@ -348,7 +347,7 @@ public void process(WatchedEvent event) { try { // am I the next leader? checkIfIamLeader(context, true); - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { } catch (Exception e) { if (!zkClient.isClosed()) { diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java index bd91a226a5d..378b2ff7043 100644 --- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java +++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java @@ -316,7 +316,7 @@ public void run() { // the workQueue is empty now, use stateUpdateQueue as fallback queue fallbackQueue = stateUpdateQueue; fallbackQueueSize = 0; - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { return; } catch (KeeperException.SessionExpiredException e) { log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e); @@ -342,7 +342,7 @@ public void run() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); return; - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { } catch (Exception e) { log.error("Exception in Overseer main queue loop", e); @@ -402,7 +402,7 @@ public void run() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); return; - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { } catch (Exception e) { log.error("Exception in Overseer main queue loop", e); @@ -414,7 +414,9 @@ public void run() { log.info("Overseer Loop exiting : {}", LeaderElector.getNodeName(myId)); } // do this in a separate thread because any wait is interrupted in this main thread - new Thread(this::checkIfIamStillLeader, "OverseerExitThread").start(); + Thread checkLeaderThread = new Thread(this::checkIfIamStillLeader, "OverseerExitThread"); + checkLeaderThread.setDaemon(true); + checkLeaderThread.start(); } } @@ -480,7 +482,7 @@ private void checkIfIamStillLeader() { byte[] data; try { data = zkClient.getData(path, null, stat, true); - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { return; } catch (Exception e) { log.warn("Error communicating with ZooKeeper", e); @@ -634,7 +636,7 @@ private LeaderStatus amILeader() { } catch (InterruptedException e) { success = false; Thread.currentThread().interrupt(); - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { success = false; } catch (Exception e) { success = false; @@ -1047,11 +1049,7 @@ ZkDistributedQueue getOverseerQuitNotificationQueue() { */ ZkDistributedQueue getStateUpdateQueue(Stats zkStats) { return new ZkDistributedQueue( - reader.getZkClient(), - "/overseer/queue", - zkStats, - STATE_UPDATE_MAX_QUEUE, - () -> Overseer.this.isClosed() || zkController.getCoreContainer().isShutDown()); + reader.getZkClient(), "/overseer/queue", zkStats, STATE_UPDATE_MAX_QUEUE); } /** diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java index eb5661885c4..b63afbcf0c4 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java @@ -36,7 +36,6 @@ import java.util.function.Predicate; import org.apache.solr.cloud.Overseer.LeaderStatus; import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent; -import org.apache.solr.common.AlreadyClosedException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkStateReader; @@ -188,7 +187,7 @@ public void run() { // We don't need to handle this. This is just a fail-safe which comes in handy in skipping // already processed async calls. log.error("KeeperException", e); - } catch (AlreadyClosedException e) { + } catch (IllegalStateException ignore) { return; } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -202,7 +201,7 @@ public void run() { try { prioritizer.prioritizeOverseerNodes(myId); - } catch (AlreadyClosedException e) { + } catch (IllegalStateException ignore) { return; } catch (Exception e) { if (!zkStateReader.getZkClient().isClosed()) { @@ -395,7 +394,7 @@ public void run() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); return; - } catch (AlreadyClosedException ignore) { + } catch (IllegalStateException ignore) { } catch (Exception e) { log.error("Exception processing", e); @@ -602,7 +601,7 @@ public void run() { response.getResponse()); } success = true; - } catch (AlreadyClosedException ignore) { + } catch (IllegalStateException ignore) { } catch (KeeperException e) { log.error("KeeperException", e); @@ -617,7 +616,7 @@ public void run() { // Reset task from tracking data structures so that it can be retried. try { resetTaskWithException(messageHandler, head.getId(), asyncId, taskKey, message); - } catch (AlreadyClosedException ignore) { + } catch (IllegalStateException ignore) { } } diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java index 6d97d785b0e..7f4548b6912 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java +++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java @@ -19,6 +19,8 @@ import java.lang.invoke.MethodHandles; import java.util.List; +import org.apache.curator.framework.api.transaction.CuratorTransactionResult; +import org.apache.curator.framework.api.transaction.OperationType; import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; @@ -36,11 +38,6 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NodeExistsException; -import org.apache.zookeeper.Op; -import org.apache.zookeeper.OpResult; -import org.apache.zookeeper.OpResult.SetDataResult; -import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,10 +103,11 @@ public void cancelElection() throws InterruptedException, KeeperException { leaderPath, leaderZkNodeParentVersion); String parent = ZkMaintenanceUtils.getZkParent(leaderPath); - List ops = - List.of(Op.check(parent, leaderZkNodeParentVersion), Op.delete(leaderPath, -1)); - zkClient.multi(ops, true); + zkClient.multi( + op -> op.check().withVersion(leaderZkNodeParentVersion).forPath(parent), + op -> op.delete().withVersion(-1).forPath(leaderPath)); } catch (InterruptedException e) { + Thread.currentThread().interrupt(); throw e; } catch (IllegalArgumentException e) { log.error("Illegal argument", e); @@ -144,27 +142,25 @@ void runLeaderProcess(boolean weAreReplacement, int pauseBeforeStartMs) // be used to make sure we only remove our own leader registration node. // The setData call used to get the parent version is also the trigger to // increment the version. We also do a sanity check that our leaderSeqPath exists. - List ops = - List.of( - Op.check(leaderSeqPath, -1), - Op.create( - leaderPath, - Utils.toJSON(leaderProps), - zkClient.getZkACLProvider().getACLsToAdd(leaderPath), - CreateMode.EPHEMERAL), - Op.setData(parent, null, -1)); - List results; - - results = zkClient.multi(ops, true); - for (OpResult result : results) { - if (result.getType() == ZooDefs.OpCode.setData) { - SetDataResult dresult = (SetDataResult) result; - Stat stat = dresult.getStat(); - leaderZkNodeParentVersion = stat.getVersion(); - return; - } - } - assert leaderZkNodeParentVersion != null; + List results = + zkClient.multi( + op -> op.check().withVersion(-1).forPath(leaderSeqPath), + op -> + op.create() + .withMode(CreateMode.EPHEMERAL) + .forPath(leaderPath, Utils.toJSON(leaderProps)), + op -> op.setData().withVersion(-1).forPath(parent, null)); + leaderZkNodeParentVersion = + results.stream() + .filter( + CuratorTransactionResult.ofTypeAndPath(OperationType.SET_DATA, parent)) + .findFirst() + .orElseThrow( + () -> + new RuntimeException( + "Could not set data for parent path in ZK: " + parent)) + .getResultStat() + .getVersion(); } }); } catch (NoNodeException e) { diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java index 33732ea0a94..59faced1b84 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -52,6 +52,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.curator.framework.api.ACLProvider; import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.cloud.SolrCloudManager; import org.apache.solr.client.solrj.impl.CloudHttp2SolrClient; @@ -68,7 +69,6 @@ import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.cloud.ClusterState; -import org.apache.solr.common.cloud.DefaultConnectionStrategy; import org.apache.solr.common.cloud.DefaultZkACLProvider; import org.apache.solr.common.cloud.DefaultZkCredentialsInjector; import org.apache.solr.common.cloud.DefaultZkCredentialsProvider; @@ -84,7 +84,6 @@ import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkACLProvider; -import org.apache.solr.common.cloud.ZkClientConnectionStrategy; import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCredentialsInjector; import org.apache.solr.common.cloud.ZkCredentialsProvider; @@ -122,7 +121,6 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.SessionExpiredException; -import org.apache.zookeeper.Op; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.ACL; @@ -233,6 +231,8 @@ public String toString() { private boolean genericCoreNodeNames; + private int clientTimeout; + private volatile boolean isClosed; private final ConcurrentHashMap replicasMetTragicEvent = @@ -329,12 +329,6 @@ public ZkController( this.leaderVoteWait = cloudConfig.getLeaderVoteWait(); this.leaderConflictResolveWait = cloudConfig.getLeaderConflictResolveWait(); - int clientTimeout = cloudConfig.getZkClientTimeout(); - - String connectionStrategy = System.getProperty("solr.zookeeper.connectionStrategy"); - ZkClientConnectionStrategy strat = - ZkClientConnectionStrategy.forName(connectionStrategy, new DefaultConnectionStrategy()); - String zkCredentialsInjectorClass = cloudConfig.getZkCredentialsInjectorClass(); ZkCredentialsInjector zkCredentialsInjector = StrUtils.isNullOrEmpty(zkCredentialsInjectorClass) @@ -342,6 +336,8 @@ public ZkController( : cc.getResourceLoader() .newInstance(zkCredentialsInjectorClass, ZkCredentialsInjector.class); + this.clientTimeout = cloudConfig.getZkClientTimeout(); + String zkACLProviderClass = cloudConfig.getZkACLProviderClass(); ZkACLProvider zkACLProvider = StrUtils.isNullOrEmpty(zkACLProviderClass) @@ -357,7 +353,6 @@ public ZkController( .newInstance(zkCredentialsProviderClass, ZkCredentialsProvider.class); zkCredentialsProvider.setZkCredentialsInjector(zkCredentialsInjector); - strat.setZkCredentialsToAddAutomatically(zkCredentialsProvider); addOnReconnectListener(getConfigDirListener()); String stateCompressionProviderClass = cloudConfig.getStateCompressorClass(); @@ -371,7 +366,6 @@ public ZkController( .withUrl(zkServerAddress) .withTimeout(clientTimeout, TimeUnit.MILLISECONDS) .withConnTimeOut(zkClientConnectTimeout, TimeUnit.MILLISECONDS) - .withConnStrategy(strat) .withReconnectListener(() -> onReconnect(descriptorsSupplier)) .withBeforeConnect(() -> beforeReconnect(descriptorsSupplier)) .withAclProvider(zkACLProvider) @@ -419,8 +413,7 @@ private void beforeReconnect(Supplier> descriptorsSupplier) markAllAsNotLeader(descriptorsSupplier); } - private void onReconnect(Supplier> descriptorsSupplier) - throws SessionExpiredException { + private void onReconnect(Supplier> descriptorsSupplier) { // on reconnect, reload cloud info log.info("ZooKeeper session re-connected ... refreshing core states after session expiration."); clearZkCollectionTerms(); @@ -521,8 +514,6 @@ private void onReconnect(Supplier> descriptorsSupplier) // Restore the interrupted status Thread.currentThread().interrupt(); throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "", e); - } catch (SessionExpiredException e) { - throw e; } catch (Exception e) { log.error("Exception during reconnect", e); throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "", e); @@ -710,8 +701,10 @@ public void preClose() { this.isClosed = true; try { - this.removeEphemeralLiveNode(); - } catch (AlreadyClosedException + if (getZkClient().isConnected()) { + this.removeEphemeralLiveNode(); + } + } catch (IllegalStateException | SessionExpiredException | KeeperException.ConnectionLossException e) { @@ -720,7 +713,7 @@ public void preClose() { } try { - if (getZkClient().getConnectionManager().isConnected()) { + if (getZkClient().isConnected()) { log.info("Publish this node as DOWN..."); publishNodeAsDown(getNodeName()); } @@ -951,12 +944,12 @@ public static void createClusterZkNodes(SolrZkClient zkClient) private static void repairSecurityJson(SolrZkClient zkClient) throws KeeperException, InterruptedException { List securityConfAcl = zkClient.getACL(ZkStateReader.SOLR_SECURITY_CONF_PATH, null, true); - ZkACLProvider aclProvider = zkClient.getZkACLProvider(); + ACLProvider aclProvider = zkClient.getZkACLProvider(); boolean tryUpdate = false; if (OPEN_ACL_UNSAFE.equals(securityConfAcl)) { - List aclToAdd = aclProvider.getACLsToAdd(ZkStateReader.SOLR_SECURITY_CONF_PATH); + List aclToAdd = aclProvider.getAclForPath(ZkStateReader.SOLR_SECURITY_CONF_PATH); if (OPEN_ACL_UNSAFE.equals(aclToAdd)) { log.warn( "Contents of zookeeper /security.json are world-readable;" @@ -966,7 +959,7 @@ private static void repairSecurityJson(SolrZkClient zkClient) } } else if (aclProvider instanceof SecurityAwareZkACLProvider) { // Use Set to explicitly ignore order - Set nonSecureACL = new HashSet<>(aclProvider.getACLsToAdd(null)); + Set nonSecureACL = new HashSet<>(aclProvider.getDefaultAcl()); // case where security.json was not treated as a secure path if (nonSecureACL.equals(new HashSet<>(securityConfAcl))) { tryUpdate = true; @@ -1070,7 +1063,7 @@ private void checkForExistingEphemeralNode() throws KeeperException, Interrupted } boolean deleted = - deletedLatch.await(zkClient.getZooKeeper().getSessionTimeout() * 2L, TimeUnit.MILLISECONDS); + deletedLatch.await(zkClient.getZkSessionTimeout() * 2L, TimeUnit.MILLISECONDS); if (!deleted) { throw new SolrException( ErrorCode.SERVER_ERROR, @@ -1188,25 +1181,19 @@ private void createEphemeralLiveNode() throws KeeperException, InterruptedExcept String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName; log.info("Register node as live in ZooKeeper:{}", nodePath); Map roles = cc.nodeRoles.getRoles(); - List ops = new ArrayList<>(roles.size() + 1); - ops.add( - Op.create( - nodePath, - null, - zkClient.getZkACLProvider().getACLsToAdd(nodePath), - CreateMode.EPHEMERAL)); + List ops = new ArrayList<>(roles.size() + 1); + ops.add(op -> op.create().withMode(CreateMode.EPHEMERAL).forPath(nodePath)); // Create the roles node as well roles.forEach( (role, mode) -> ops.add( - Op.create( - NodeRoles.getZNodeForRoleMode(role, mode) + "/" + nodeName, - null, - zkClient.getZkACLProvider().getACLsToAdd(nodePath), - CreateMode.EPHEMERAL))); + op -> + op.create() + .withMode(CreateMode.EPHEMERAL) + .forPath(NodeRoles.getZNodeForRoleMode(role, mode) + "/" + nodeName))); - zkClient.multi(ops, true); + zkClient.multi(ops); } public void removeEphemeralLiveNode() throws KeeperException, InterruptedException { @@ -1216,10 +1203,8 @@ public void removeEphemeralLiveNode() throws KeeperException, InterruptedExcepti String nodeName = getNodeName(); String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName; log.info("Remove node as live in ZooKeeper:{}", nodePath); - List ops = List.of(Op.delete(nodePath, -1)); - try { - zkClient.multi(ops, true); + zkClient.delete(nodePath, -1, true); } catch (NoNodeException e) { } @@ -1511,7 +1496,7 @@ private String getLeader(final CloudDescriptor cloudDesc, int timeoutms) { leaderUrl = getLeaderProps(collection, cloudDesc.getShardId(), timeoutms).getCoreUrl(); } - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { throw e; } catch (Exception e) { log.error("Error getting leader from zk", e); @@ -2539,7 +2524,7 @@ public static int persistConfigResourceToZooKeeper( } catch (NoNodeException e) { if (createIfNotExists) { try { - zkClient.create(resourceLocation, content, CreateMode.PERSISTENT, true); + zkClient.makePath(resourceLocation, content, CreateMode.PERSISTENT, true); latestVersion = 0; // just created so version must be zero touchConfDir(zkLoader); } catch (KeeperException.NodeExistsException nee) { @@ -2892,7 +2877,7 @@ public Collection publishNodeAsDown(String nodeName) { m -> m.put(Overseer.QUEUE_OPERATION, OverseerAction.DOWNNODE.toLower()) .put(ZkStateReader.NODE_NAME_PROP, nodeName)); - } catch (AlreadyClosedException e) { + } catch (IllegalStateException e) { log.info( "Not publishing node as DOWN because a resource required to do so is already closed."); } catch (InterruptedException e) { diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java b/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java index 57db14740c5..f3e06f90cb8 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java @@ -32,16 +32,15 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Predicate; +import org.apache.curator.framework.api.transaction.CuratorTransactionResult; import org.apache.solr.client.solrj.cloud.DistributedQueue; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; -import org.apache.solr.common.cloud.ConnectionManager.IsClosed; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkMaintenanceUtils; import org.apache.solr.common.util.Pair; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Op; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.Stat; @@ -123,15 +122,6 @@ public ZkDistributedQueue(SolrZkClient zookeeper, String dir, Stats stats) { } public ZkDistributedQueue(SolrZkClient zookeeper, String dir, Stats stats, int maxQueueSize) { - this(zookeeper, dir, stats, maxQueueSize, null); - } - - public ZkDistributedQueue( - SolrZkClient zookeeper, - String dir, - Stats stats, - int maxQueueSize, - IsClosed higherLevelIsClosed) { this.dir = dir; try { @@ -246,24 +236,19 @@ public byte[] remove() throws NoSuchElementException, KeeperException, Interrupt public void remove(Collection paths) throws KeeperException, InterruptedException { if (paths.isEmpty()) return; - List ops = new ArrayList<>(); + List ops = new ArrayList<>(); for (String path : paths) { - ops.add(Op.delete(dir + "/" + path, -1)); + ops.add(op -> op.delete().withVersion(-1).forPath(dir + "/" + path)); } for (int from = 0; from < ops.size(); from += 1000) { int to = Math.min(from + 1000, ops.size()); if (from < to) { - try { - zookeeper.multi(ops.subList(from, to), true); - } catch (KeeperException.NoNodeException e) { - // don't know which nodes are not exist, so try to delete one by one node - for (int j = from; j < to; j++) { + Collection results = zookeeper.multi(ops.subList(from, to)); + for (CuratorTransactionResult result : results) { + if (result.getError() != 0) { try { - zookeeper.delete(ops.get(j).getPath(), -1, true); - } catch (KeeperException.NoNodeException e2) { - if (log.isDebugEnabled()) { - log.debug("Can not remove node which is not exist : {}", ops.get(j).getPath()); - } + zookeeper.delete(result.getForPath(), -1, true); + } catch (KeeperException.NoNodeException ignored) { } } } diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java index b15567bf082..83f8885325b 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java @@ -22,7 +22,6 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -398,16 +397,6 @@ private void retryRegisterWatcher() { return; } catch (KeeperException e) { log.warn("Failed watching shard term for collection: {}, retrying!", collection, e); - try { - zkClient.getConnectionManager().waitForConnected(zkClient.getZkClientTimeout()); - } catch (TimeoutException te) { - if (Thread.interrupted()) { - throw new SolrException( - SolrException.ErrorCode.SERVER_ERROR, - "Error watching shard term for collection: " + collection, - te); - } - } } } } @@ -416,6 +405,10 @@ private void retryRegisterWatcher() { private void registerWatcher() throws KeeperException { Watcher watcher = event -> { + // Don't do anything if we are closed + if (isClosed.get()) { + return; + } // session events are not change events, and do not remove the watcher if (Watcher.Event.EventType.None == event.getType()) { return; diff --git a/solr/core/src/java/org/apache/solr/core/ZkContainer.java b/solr/core/src/java/org/apache/solr/core/ZkContainer.java index 02e6d4e628b..6dd7b2d571d 100644 --- a/solr/core/src/java/org/apache/solr/core/ZkContainer.java +++ b/solr/core/src/java/org/apache/solr/core/ZkContainer.java @@ -32,7 +32,6 @@ import org.apache.solr.client.solrj.impl.SolrZkClientTimeout; import org.apache.solr.cloud.SolrZkServer; import org.apache.solr.cloud.ZkController; -import org.apache.solr.common.AlreadyClosedException; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.ClusterProperties; import org.apache.solr.common.cloud.Replica; @@ -217,7 +216,7 @@ public void registerInZk(final SolrCore core, boolean background, boolean skipRe log.error("Interrupted", e); } catch (KeeperException e) { log.error("KeeperException registering core {}", core.getName(), e); - } catch (AlreadyClosedException ignore) { + } catch (IllegalStateException ignore) { } catch (Exception e) { log.error("Exception registering core {}", core.getName(), e); diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java index 28db4f4478c..2c1503a907f 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java @@ -323,9 +323,7 @@ public ShardHandler getAndPrepShardHandler(SolrQueryRequest req, ResponseBuilder boolean requireZkConnected = shardsTolerant != null && shardsTolerant.equals(ShardParams.REQUIRE_ZK_CONNECTED); ZkController zkController = cc.getZkController(); - boolean zkConnected = - zkController != null - && !zkController.getZkClient().getConnectionManager().isLikelyExpired(); + boolean zkConnected = zkController != null && zkController.getZkClient().isConnected(); if (requireZkConnected && false == zkConnected) { throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ZooKeeper is not connected"); } else { diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java index 35d96a7c14a..8c8c3ce6a08 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java +++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java @@ -1394,7 +1394,7 @@ private void zkCheck(UpdateCommand updateCommand) { return; } - if (!zkController.getZkClient().getConnectionManager().isLikelyExpired()) { + if (zkController.isConnected()) { return; } diff --git a/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java b/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java index 5cf76769d99..302741b72d1 100644 --- a/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java +++ b/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java @@ -178,7 +178,7 @@ public void testPutCompressed() throws Exception { ZkCpTool tool = new ZkCpTool(); assertEquals(0, runTool(args, tool)); - assertArrayEquals(zkClient.getZooKeeper().getData("/state.json", null, null), expected); + assertArrayEquals(zkClient.getCuratorFramework().getData().forPath("/state.json"), expected); // test re-put to existing data = "my data deux"; @@ -201,13 +201,13 @@ public void testPutCompressed() throws Exception { }; assertEquals(0, runTool(args, tool)); - byte[] fromZk = zkClient.getZooKeeper().getData("/state.json", null, null); + byte[] fromZk = zkClient.getCuratorFramework().getData().forPath("/state.json"); byte[] fromLoc = new ZLibCompressor() .compressBytes(Files.readAllBytes(Path.of(localFile.getAbsolutePath()))); assertArrayEquals("Should get back what we put in ZK", fromLoc, fromZk); - assertArrayEquals(zkClient.getZooKeeper().getData("/state.json", null, null), expected); + assertArrayEquals(zkClient.getCuratorFramework().getData().forPath("/state.json"), expected); } @Test @@ -272,7 +272,7 @@ public void testPutFileCompressed() throws Exception { ZkCpTool tool = new ZkCpTool(); assertEquals(0, runTool(args, tool)); - byte[] fromZk = zkClient.getZooKeeper().getData("/state.json", null, null); + byte[] fromZk = zkClient.getCuratorFramework().getData().forPath("/state.json"); Path locFile = Path.of(SOLR_HOME, "solr-stress-new.xml"); byte[] fromLoc = new ZLibCompressor().compressBytes(Files.readAllBytes(locFile)); assertArrayEquals("Should get back what we put in ZK", fromLoc, fromZk); @@ -280,7 +280,7 @@ public void testPutFileCompressed() throws Exception { // Lets do it again assertEquals(0, runTool(args, tool)); - fromZk = zkClient.getZooKeeper().getData("/state.json", null, null); + fromZk = zkClient.getCuratorFramework().getData().forPath("/state.json"); locFile = Path.of(SOLR_HOME, "solr-stress-new.xml"); fromLoc = new ZLibCompressor().compressBytes(Files.readAllBytes(locFile)); assertArrayEquals("Should get back what we put in ZK", fromLoc, fromZk); diff --git a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java deleted file mode 100644 index d48ba5bc3cc..00000000000 --- a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * 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.cloud; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.common.cloud.ConnectionManager; -import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.common.util.SolrNamedThreadFactory; -import org.apache.zookeeper.TestableZooKeeper; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.Watcher.Event.EventType; -import org.apache.zookeeper.Watcher.Event.KeeperState; -import org.apache.zookeeper.ZooKeeper; -import org.junit.Ignore; -import org.junit.Test; - -public class ConnectionManagerTest extends SolrTestCaseJ4 { - - static final int TIMEOUT = 3000; - - @Ignore - public void testConnectionManager() throws Exception { - - // setup a SolrZkClient to do some getBaseUrlForNodeName testing - Path zkDir = createTempDir("zkData"); - ZkTestServer server = new ZkTestServer(zkDir); - try { - server.run(); - - SolrZkClient zkClient = - new SolrZkClient.Builder() - .withUrl(server.getZkAddress()) - .withTimeout(TIMEOUT, TimeUnit.MILLISECONDS) - .build(); - ConnectionManager cm = zkClient.getConnectionManager(); - try { - assertFalse(cm.isLikelyExpired()); - - ZooKeeper zk = zkClient.getZooKeeper(); - assertTrue(zk instanceof TestableZooKeeper); - ((TestableZooKeeper) zk).testableConnloss(); - server.expire(zkClient.getZooKeeper().getSessionId()); - - Thread.sleep(TIMEOUT); - - assertTrue(cm.isLikelyExpired()); - } finally { - cm.close(); - zkClient.close(); - } - } finally { - server.shutdown(); - } - } - - public void testLikelyExpired() throws Exception { - - // setup a SolrZkClient to do some getBaseUrlForNodeName testing - Path zkDir = createTempDir("zkData"); - ZkTestServer server = new ZkTestServer(zkDir); - try { - server.run(); - - SolrZkClient zkClient = - new SolrZkClient.Builder() - .withUrl(server.getZkAddress()) - .withTimeout(TIMEOUT, TimeUnit.MILLISECONDS) - .build(); - ConnectionManager cm = zkClient.getConnectionManager(); - try { - assertFalse(cm.isLikelyExpired()); - assertTrue(cm.isConnectedAndNotClosed()); - cm.process(new WatchedEvent(EventType.None, KeeperState.Disconnected, "")); - // disconnect shouldn't immediately set likelyExpired - assertFalse(cm.isConnectedAndNotClosed()); - assertFalse(cm.isLikelyExpired()); - - // but it should after the timeout - Thread.sleep((long) (zkClient.getZkClientTimeout() * 1.5)); - assertFalse(cm.isConnectedAndNotClosed()); - assertTrue(cm.isLikelyExpired()); - - // even if we disconnect immediately again - cm.process(new WatchedEvent(EventType.None, KeeperState.Disconnected, "")); - assertFalse(cm.isConnectedAndNotClosed()); - assertTrue(cm.isLikelyExpired()); - - // reconnect -- should no longer be likely expired - cm.process(new WatchedEvent(EventType.None, KeeperState.SyncConnected, "")); - assertFalse(cm.isLikelyExpired()); - assertTrue(cm.isConnectedAndNotClosed()); - } finally { - cm.close(); - zkClient.close(); - } - } finally { - server.shutdown(); - } - } - - @Test - public void testReconnectWhenZkDisappeared() throws Exception { - ScheduledExecutorService executor = - Executors.newSingleThreadScheduledExecutor( - new SolrNamedThreadFactory("connectionManagerTest")); - - // setup a SolrZkClient to do some getBaseUrlForNodeName testing - Path zkDir = createTempDir("zkData"); - ZkTestServer server = new ZkTestServer(zkDir); - try { - server.run(); - - MockZkClientConnectionStrategy strategy = new MockZkClientConnectionStrategy(); - SolrZkClient zkClient = - new SolrZkClient.Builder() - .withUrl(server.getZkAddress()) - .withTimeout(TIMEOUT, TimeUnit.MILLISECONDS) - .withConnStrategy(strategy) - .build(); - ConnectionManager cm = zkClient.getConnectionManager(); - - try { - assertFalse(cm.isLikelyExpired()); - assertTrue(cm.isConnectedAndNotClosed()); - - // reconnect -- should no longer be likely expired - cm.process(new WatchedEvent(EventType.None, KeeperState.Expired, "")); - assertFalse(cm.isLikelyExpired()); - assertTrue(cm.isConnectedAndNotClosed()); - assertTrue(strategy.isExceptionThrow()); - } finally { - cm.close(); - zkClient.close(); - executor.shutdown(); - } - } finally { - server.shutdown(); - } - } - - private static class MockZkClientConnectionStrategy extends TestConnectionStrategy { - int called = 0; - boolean exceptionThrown = false; - - @Override - public void reconnect( - final String serverAddress, - final int zkClientTimeout, - final Watcher watcher, - final ZkUpdate updater) - throws IOException, InterruptedException, TimeoutException { - - if (called++ < 1) { - exceptionThrown = true; - throw new IOException("Testing"); - } - - super.reconnect(serverAddress, zkClientTimeout, watcher, updater); - } - - public boolean isExceptionThrow() { - return exceptionThrown; - } - } -} diff --git a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java index 9a57f4d6c9d..213a5db6d6d 100644 --- a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java @@ -19,6 +19,7 @@ import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -26,6 +27,7 @@ import java.util.function.Predicate; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.client.solrj.cloud.DistributedQueue; +import org.apache.solr.common.cloud.OnDisconnect; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.SolrNamedThreadFactory; @@ -289,18 +291,19 @@ public void testPeekElements() throws Exception { } private void forceSessionExpire() throws InterruptedException, TimeoutException { - long sessionId = zkClient.getZooKeeper().getSessionId(); + final CountDownLatch hasDisconnected = new CountDownLatch(1); + zkClient + .getCuratorFramework() + .getConnectionStateListenable() + .addListener((OnDisconnect) hasDisconnected::countDown); + long sessionId = zkClient.getZkSessionId(); zkServer.expire(sessionId); - zkClient.getConnectionManager().waitForDisconnected(10000); - zkClient.getConnectionManager().waitForConnected(10000); - for (int i = 0; i < 100; ++i) { - if (zkClient.isConnected()) { - break; - } - Thread.sleep(50); - } + hasDisconnected.await(10, TimeUnit.SECONDS); + assertEquals( + "ZK Client did not disconnect after session expiration", 0, hasDisconnected.getCount()); + zkClient.getCuratorFramework().blockUntilConnected(10, TimeUnit.SECONDS); assertTrue(zkClient.isConnected()); - assertNotEquals(sessionId, zkClient.getZooKeeper().getSessionId()); + assertNotEquals(sessionId, zkClient.getZkSessionId()); } protected ZkDistributedQueue makeDistributedQueue(String dqZNode) throws Exception { diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java index 44e226afdf0..d4a61abcc60 100644 --- a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java @@ -28,6 +28,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.curator.test.KillSession; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.cloud.OnReconnect; import org.apache.solr.common.cloud.SolrZkClient; @@ -39,8 +40,6 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.SessionExpiredException; -import org.apache.zookeeper.TestableZooKeeper; -import org.apache.zookeeper.ZooKeeper; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,7 +118,7 @@ public ElectorSetup(OnReconnect onReconnect) { .build(); zkStateReader = new ZkStateReader(zkClient); elector = new LeaderElector(zkClient); - zkController = MockSolrSource.makeSimpleMock(null, zkStateReader, null); + zkController = MockSolrSource.makeSimpleMock(null, zkStateReader, zkClient); } public void close() { @@ -187,9 +186,10 @@ private void setupOnConnect() throws InterruptedException, KeeperException { public void run() { try { setupOnConnect(); + } catch (IllegalStateException ignored) { + return; } catch (Throwable e) { log.error("setup failed", e); - es.close(); return; } @@ -478,8 +478,7 @@ public void testStressElection() throws Exception { final List threads = Collections.synchronizedList(new ArrayList()); // start with a leader - ClientThread thread1 = null; - thread1 = new ClientThread("shard1", 0); + ClientThread thread1 = new ClientThread("shard1", 0); threads.add(thread1); scheduler.schedule(thread1, 0, TimeUnit.MILLISECONDS); @@ -539,11 +538,15 @@ public void run() { int j; j = random().nextInt(threads.size()); try { - ZooKeeper zk = threads.get(j).es.zkClient.getZooKeeper(); - assertTrue(zk instanceof TestableZooKeeper); - ((TestableZooKeeper) zk).testableConnloss(); if (random().nextBoolean()) { - server.expire(zk.getSessionId()); + KillSession.kill( + threads + .get(j) + .es + .zkClient + .getCuratorFramework() + .getZookeeperClient() + .getZooKeeper()); } } catch (Exception e) { log.error("error expiring session", e); @@ -555,36 +558,43 @@ public void run() { } }); - scheduleThread.start(); - connLossThread.start(); - killThread.start(); + try { + scheduleThread.start(); + connLossThread.start(); + killThread.start(); - Thread.sleep(4000); + Thread.sleep(4000); - stopStress = true; + stopStress = true; - scheduleThread.interrupt(); - connLossThread.interrupt(); - killThread.interrupt(); + scheduleThread.interrupt(); + connLossThread.interrupt(); + killThread.interrupt(); - scheduleThread.join(); - scheduler.shutdownNow(); + scheduleThread.join(); + scheduler.shutdownNow(); - connLossThread.join(); - killThread.join(); + connLossThread.join(); + killThread.join(); - int seq = threads.get(getLeaderThread()).getSeq(); + int seq = threads.get(getLeaderThread()).getSeq(); - // we have a leader we know, TODO: lets check some other things + // we have a leader we know, TODO: lets check some other things + } finally { + // cleanup any threads still running + for (ClientThread thread : threads) { + thread.close(); + } - // cleanup any threads still running - for (ClientThread thread : threads) { - thread.es.zkClient.getZooKeeper().close(); - thread.close(); - } + // cleanup any threads still running + for (ClientThread thread : threads) { + thread.es.zkClient.close(); + thread.close(); + } - for (Thread thread : threads) { - thread.join(); + for (Thread thread : threads) { + thread.join(); + } } } diff --git a/solr/core/src/test/org/apache/solr/cloud/MultiThreadedOCPTest.java b/solr/core/src/test/org/apache/solr/cloud/MultiThreadedOCPTest.java index e67fa4f280e..833f28a1a63 100644 --- a/solr/core/src/test/org/apache/solr/cloud/MultiThreadedOCPTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/MultiThreadedOCPTest.java @@ -246,8 +246,7 @@ public void testTaskExclusivity() throws Exception, SolrServerException { @Test public void testDeduplicationOfSubmittedTasks() throws IOException, SolrServerException { try (SolrClient client = createNewSolrClient("", getBaseUrl(jettys.get(0)))) { - CollectionAdminRequest.createCollection("ocptest_shardsplit2", "conf1", 3, 1) - .processAsync("3000", client); + CollectionAdminRequest.createCollection("ocptest_shardsplit2", "conf1", 3, 1).process(client); SplitShard splitShardRequest = CollectionAdminRequest.splitShard("ocptest_shardsplit2").setShardName(SHARD1); diff --git a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java index 8c45324946c..b0188069562 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java @@ -152,7 +152,7 @@ public void testOpenACLUnsafeAllover() throws Exception { protected void assertOpenACLUnsafeAllover( SolrZkClient zkClient, String path, List verifiedList) throws Exception { - List acls = zkClient.getZooKeeper().getACL(path, new Stat()); + List acls = zkClient.getACL(path, new Stat(), true); if (log.isInfoEnabled()) { log.info("Verifying {}", path); } diff --git a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java index 79fdac41acf..5810c50fe17 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java @@ -29,15 +29,17 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; +import org.apache.curator.framework.AuthInfo; +import org.apache.curator.framework.api.ACLProvider; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.cloud.DigestZkACLProvider; import org.apache.solr.common.cloud.DigestZkCredentialsProvider; import org.apache.solr.common.cloud.SecurityAwareZkACLProvider; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.VMParamsZkCredentialsInjector; -import org.apache.solr.common.cloud.ZkACLProvider; import org.apache.solr.common.cloud.ZkCredentialsInjector; import org.apache.solr.common.cloud.ZkCredentialsProvider; +import org.apache.solr.common.util.StrUtils; import org.apache.zookeeper.CreateMode; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -80,50 +82,57 @@ public void setUp() throws Exception { System.setProperty("zkHost", zkServer.getZkAddress()); - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders( "connectAndAllACLUsername", "connectAndAllACLPassword", "readonlyACLUsername", "readonlyACLPassword") - .getSolrZkClient(zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT); - zkClient.makePath("/solr", false, true); - zkClient.close(); + .getSolrZkClient(zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT)) { + zkClient.makePath("/solr", false, true); + } - zkClient = + try (SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders( "connectAndAllACLUsername", "connectAndAllACLPassword", "readonlyACLUsername", "readonlyACLPassword") - .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT); - zkClient.create( - "/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false); - zkClient.makePath( - "/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false); - zkClient.create( - SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, - "content".getBytes(DATA_ENCODING), - CreateMode.PERSISTENT, - false); - zkClient.close(); - - zkClient = + .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT)) { + zkClient.create( + "/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false); + zkClient.makePath( + "/protectedMakePathNode", + "content".getBytes(DATA_ENCODING), + CreateMode.PERSISTENT, + false); + zkClient.create( + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, + "content".getBytes(DATA_ENCODING), + CreateMode.PERSISTENT, + false); + } + + try (SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders(null, null, null, null) - .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT); - zkClient - .getZooKeeper() - .addAuthInfo( - "digest", - ("connectAndAllACLUsername:connectAndAllACLPassword").getBytes(DATA_ENCODING)); - zkClient.create( - "/unprotectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false); - zkClient.makePath( - "/unprotectedMakePathNode", - "content".getBytes(DATA_ENCODING), - CreateMode.PERSISTENT, - false); - zkClient.close(); + .getSolrZkClient( + zkServer.getZkAddress(), + AbstractZkTestCase.TIMEOUT, + new AuthInfo( + "digest", + ("connectAndAllACLUsername:connectAndAllACLPassword") + .getBytes(DATA_ENCODING)))) { + zkClient.create( + "/unprotectedCreateNode", + "content".getBytes(DATA_ENCODING), + CreateMode.PERSISTENT, + false); + zkClient.makePath( + "/unprotectedMakePathNode", + "content".getBytes(DATA_ENCODING), + CreateMode.PERSISTENT, + false); + } if (log.isInfoEnabled()) { log.info("####SETUP_END {}", getTestName()); @@ -141,58 +150,46 @@ public void tearDown() throws Exception { @Test public void testNoCredentialsSolrZkClientFactoryUsingCompletelyNewProviders() throws Exception { - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders(null, null, null, null) - .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT); - try { + .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT)) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, false, false, false, false, false, false, false, false, false, false); - } finally { - zkClient.close(); } } @Test public void testWrongCredentialsSolrZkClientFactoryUsingCompletelyNewProviders() throws Exception { - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders( "connectAndAllACLUsername", "connectAndAllACLPasswordWrong", null, null) - .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT); - try { + .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT)) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, false, false, false, false, false, false, false, false, false, false); - } finally { - zkClient.close(); } } @Test public void testAllCredentialsSolrZkClientFactoryUsingCompletelyNewProviders() throws Exception { - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders( "connectAndAllACLUsername", "connectAndAllACLPassword", null, null) - .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT); - try { + .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT)) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, true, true, true, true, true, true, true, true, true, true); - } finally { - zkClient.close(); } } @Test public void testReadonlyCredentialsSolrZkClientFactoryUsingCompletelyNewProviders() throws Exception { - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders( "readonlyACLUsername", "readonlyACLPassword", null, null) - .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT); - try { + .getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT)) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, true, true, false, false, false, false, false, false, false, false); - } finally { - zkClient.close(); } } @@ -202,16 +199,13 @@ public void testReadonlyCredentialsSolrZkClientFactoryUsingCompletelyNewProvider throws Exception { useNoCredentials(); - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames( new SolrZkClient.Builder() .withUrl(zkServer.getZkAddress()) - .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)); - try { + .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS))) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, false, false, false, false, false, false, false, false, false, false); - } finally { - zkClient.close(); } } @@ -221,16 +215,13 @@ public void testReadonlyCredentialsSolrZkClientFactoryUsingCompletelyNewProvider throws Exception { useWrongCredentials(); - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames( new SolrZkClient.Builder() .withUrl(zkServer.getZkAddress()) - .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)); - try { + .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS))) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, false, false, false, false, false, false, false, false, false, false); - } finally { - zkClient.close(); } } @@ -240,16 +231,13 @@ public void testReadonlyCredentialsSolrZkClientFactoryUsingCompletelyNewProvider throws Exception { useAllCredentials(); - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames( new SolrZkClient.Builder() .withUrl(zkServer.getZkAddress()) - .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)); - try { + .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS))) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, true, true, true, true, true, true, true, true, true, true); - } finally { - zkClient.close(); } } @@ -259,16 +247,13 @@ public void testReadonlyCredentialsSolrZkClientFactoryUsingCompletelyNewProvider throws Exception { useReadonlyCredentials(); - SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames( new SolrZkClient.Builder() .withUrl(zkServer.getZkAddress()) - .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)); - try { + .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS))) { AbstractDigestZkACLAndCredentialsProvidersTestBase.doTest( zkClient, true, true, false, false, false, false, false, false, false, false); - } finally { - zkClient.close(); } } @@ -291,35 +276,38 @@ public SolrZkClientFactoryUsingCompletelyNewProviders( this.digestReadonlyPassword = digestReadonlyPassword; zkCredentialsInjector = () -> { - List zkCredentials = new ArrayList<>(2); ZkCredentialsInjector.ZkCredential allCreds = new ZkCredentialsInjector.ZkCredential(digestUsername, digestPassword, Perms.ALL); ZkCredentialsInjector.ZkCredential readCreds = new ZkCredentialsInjector.ZkCredential( digestReadonlyUsername, digestReadonlyPassword, Perms.READ); - zkCredentials.add(allCreds); - zkCredentials.add(readCreds); - return zkCredentials; + return List.of(allCreds, readCreds); }; } public SolrZkClient getSolrZkClient(String zkServerAddress, int zkClientTimeout) { + AuthInfo authInfo = null; + if (StrUtils.isNotNullOrEmpty(digestUsername) && StrUtils.isNotNullOrEmpty(digestPassword)) { + authInfo = + new AuthInfo( + "digest", (digestUsername + ":" + digestPassword).getBytes(StandardCharsets.UTF_8)); + } + return getSolrZkClient(zkServerAddress, zkClientTimeout, authInfo); + } - return new SolrZkClient( - new SolrZkClient.Builder() - .withUrl(zkServerAddress) - .withTimeout(zkClientTimeout, TimeUnit.MILLISECONDS)) { - - @Override - protected ZkCredentialsProvider createZkCredentialsToAddAutomatically() { - return new DigestZkCredentialsProvider(zkCredentialsInjector); - } - - @Override - public ZkACLProvider createZkACLProvider() { - return new DigestZkACLProvider(zkCredentialsInjector); - } - }; + public SolrZkClient getSolrZkClient( + String zkServerAddress, int zkClientTimeout, AuthInfo authInfo) { + final List authInfos = new ArrayList<>(); + if (authInfo != null) { + authInfos.add(authInfo); + } + return new SolrZkClient.Builder() + .withUrl(zkServerAddress) + .withZkCredentialsProvider( + new DigestZkCredentialsProvider(zkCredentialsInjector, authInfos)) + .withAclProvider(new DigestZkACLProvider(zkCredentialsInjector)) + .withTimeout(zkClientTimeout, TimeUnit.MILLISECONDS) + .build(); } } @@ -349,7 +337,7 @@ protected ZkCredentialsProvider createZkCredentialsToAddAutomatically() { } @Override - public ZkACLProvider createZkACLProvider() { + public ACLProvider createACLProvider() { return new DigestZkACLProvider(vmParamsZkCredentialsInjector); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java index 6c5af8facd9..1350106457d 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java @@ -58,7 +58,6 @@ import org.apache.solr.cloud.overseer.NodeMutator; import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.cloud.overseer.ZkWriteCommand; -import org.apache.solr.common.AlreadyClosedException; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.DocCollection; @@ -1231,7 +1230,7 @@ public void testShardLeaderChange() throws Exception { ZkDistributedQueue q = getOpenOverseer().getStateUpdateQueue(); q.offer(m); break; - } catch (SolrException | KeeperException | AlreadyClosedException e) { + } catch (SolrException | KeeperException | IllegalStateException e) { log.error("error updating state", e); } } @@ -1249,7 +1248,7 @@ public void testShardLeaderChange() throws Exception { true, getOpenOverseer()); break; - } catch (SolrException | KeeperException | AlreadyClosedException e) { + } catch (SolrException | KeeperException | IllegalStateException e) { log.error("error publishing state", e); } } @@ -1274,7 +1273,7 @@ public void testShardLeaderChange() throws Exception { true, getOpenOverseer()); break; - } catch (SolrException | AlreadyClosedException e) { + } catch (SolrException | IllegalStateException e) { log.error("error publishing state", e); } } @@ -1294,7 +1293,7 @@ public void testShardLeaderChange() throws Exception { true, getOpenOverseer()); break; - } catch (SolrException | AlreadyClosedException e) { + } catch (SolrException | IllegalStateException e) { log.error("error publishing state", e); } } @@ -1314,7 +1313,7 @@ public void testShardLeaderChange() throws Exception { true, getOpenOverseer()); break; - } catch (SolrException | AlreadyClosedException e) { + } catch (SolrException | IllegalStateException e) { log.error("error publishing state", e); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java b/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java index b84d3f49e37..a871fc7176c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java @@ -69,7 +69,8 @@ public void testLeaderElectionWithZkExpiry() throws Exception { () -> { TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME); while (!timeout.hasTimedOut()) { - server.expire(zkController.getZkClient().getZooKeeper().getSessionId()); + long sessionId = zkController.getZkClient().getZkSessionId(); + server.expire(sessionId); try { timeout.sleep(10); } catch (InterruptedException e) { diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java index 9906d3041f1..f4af183434c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java @@ -51,6 +51,8 @@ import org.slf4j.LoggerFactory; @SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776") +// PRS Defaulting is currently not working, so disable for now +@SolrCloudTestCase.NoPrs public class TestPullReplicaErrorHandling extends SolrCloudTestCase { private static final int REPLICATION_TIMEOUT_SECS = 10; diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSizeLimitedDistributedMap.java b/solr/core/src/test/org/apache/solr/cloud/TestSizeLimitedDistributedMap.java index a534e2e2a30..49cecefca05 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestSizeLimitedDistributedMap.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestSizeLimitedDistributedMap.java @@ -91,8 +91,7 @@ public void testConcurrentCleanup() throws Exception { .build()) { String path = getAndMakeInitialPath(zkClient); DistributedMap map = - new SizeLimitedDistributedMap( - zkClient, path, numResponsesToStore, (element) -> deletedItems.add(element)); + new SizeLimitedDistributedMap(zkClient, path, numResponsesToStore, deletedItems::add); // fill the map to limit first for (int i = 0; i < numResponsesToStore; i++) { map.put("xyz_" + i, new byte[0]); diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java index 4503c862d50..2ce44a8b65c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java @@ -20,20 +20,16 @@ import java.io.IOException; import java.nio.file.Path; -import java.util.Arrays; import java.util.HashSet; -import java.util.List; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.common.cloud.ZkCmdExecutor; import org.apache.solr.common.cloud.ZkMaintenanceUtils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Op; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.Stat; @@ -51,8 +47,8 @@ public static void beforeClass() throws Exception { @SuppressWarnings({"try"}) static class ZkConnection implements AutoCloseable { - private ZkTestServer server = null; - private SolrZkClient zkClient = null; + private final ZkTestServer server; + private final SolrZkClient zkClient; ZkConnection() throws Exception { Path zkDir = createTempDir("zkData"); @@ -91,15 +87,12 @@ public void testConnect() throws Exception { @SuppressWarnings({"try"}) public void testMakeRootNode() throws Exception { try (ZkConnection conn = new ZkConnection()) { - final SolrZkClient zkClient = + try (SolrZkClient zkClient = new SolrZkClient.Builder() .withUrl(conn.getServer().getZkHost()) .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS) - .build(); - try { + .build()) { assertTrue(zkClient.exists("/solr", true)); - } finally { - zkClient.close(); } } } @@ -122,7 +115,7 @@ public void testClean() throws Exception { public void testReconnect() throws Exception { Path zkDir = createTempDir("zkData"); - ZkTestServer server = null; + ZkTestServer server; server = new ZkTestServer(zkDir); server.run(); try (SolrZkClient zkClient = @@ -191,7 +184,8 @@ public void run() { // simulate session expiration // one option - server.expire(zkClient.getZooKeeper().getSessionId()); + long sessionId = zkClient.getZkSessionId(); + server.expire(sessionId); // another option // zkClient.getSolrZooKeeper().getConnection().disconnect(); @@ -223,37 +217,6 @@ public void run() { } } - public void testZkCmdExecutor() throws Exception { - Path zkDir = createTempDir("zkData"); - ZkTestServer server = null; - - try { - server = new ZkTestServer(zkDir); - server.run(); - - final int timeout = random().nextInt(10000) + 5000; - - ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor(timeout); - final long start = System.nanoTime(); - expectThrows( - KeeperException.SessionExpiredException.class, - () -> { - zkCmdExecutor.retryOperation( - () -> { - if (System.nanoTime() - start - > TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS)) { - throw new KeeperException.SessionExpiredException(); - } - throw new KeeperException.ConnectionLossException(); - }); - }); - } finally { - if (server != null) { - server.shutdown(); - } - } - } - @Test @SuppressWarnings({"try"}) public void testMultipleWatchesAsync() throws Exception { @@ -444,15 +407,9 @@ public void testZkBehavior() throws Exception { Stat stat = zkClient.exists("/test-node", null, true); int cversion = stat.getCversion(); - List ops = - Arrays.asList( - Op.create( - "/test-node/abc", - null, - zkClient.getZkACLProvider().getACLsToAdd("/test-node/abc"), - CreateMode.PERSISTENT), - Op.delete("/test-node/abc", -1)); - zkClient.multi(ops, true); + zkClient.multi( + op -> op.create().withMode(CreateMode.PERSISTENT).forPath("/test-node/abc", null), + op -> op.delete().withVersion(-1).forPath("/test-node/abc")); stat = zkClient.exists("/test-node", null, true); assertTrue(stat.getCversion() >= cversion + 2); } finally { diff --git a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java index 7d05566675a..4008f694359 100644 --- a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java @@ -472,9 +472,7 @@ public void testSingleExternalCollectionCompressedState() throws Exception { writer.writePendingUpdates(); byte[] data = - zkClient - .getZooKeeper() - .getData(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", null, null); + zkClient.getData(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", null, null, true); Map map = (Map) Utils.fromJSON(data); assertNotNull(map.get("c1")); } @@ -511,8 +509,9 @@ public void testSingleExternalCollectionCompressedState() throws Exception { byte[] data = zkClient - .getZooKeeper() - .getData(ZkStateReader.COLLECTIONS_ZKNODE + "/c2/state.json", null, null); + .getCuratorFramework() + .getData() + .forPath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2/state.json"); assertTrue(compressor.isCompressedBytes(data)); Map map = (Map) Utils.fromJSON(compressor.decompressBytes(data)); assertNotNull(map.get("c2")); diff --git a/solr/licenses/curator-client-LICENSE-ASL.txt b/solr/licenses/curator-LICENSE-ASL.txt similarity index 100% rename from solr/licenses/curator-client-LICENSE-ASL.txt rename to solr/licenses/curator-LICENSE-ASL.txt diff --git a/solr/licenses/curator-client-NOTICE.txt b/solr/licenses/curator-NOTICE.txt similarity index 100% rename from solr/licenses/curator-client-NOTICE.txt rename to solr/licenses/curator-NOTICE.txt diff --git a/solr/licenses/curator-client-5.7.0.jar.sha1 b/solr/licenses/curator-client-5.7.0.jar.sha1 deleted file mode 100644 index 7bb1b27f325..00000000000 --- a/solr/licenses/curator-client-5.7.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -690e470ffe1c4bea8149d8eca9982842a447ffc6 diff --git a/solr/licenses/curator-client-5.7.1.jar.sha1 b/solr/licenses/curator-client-5.7.1.jar.sha1 new file mode 100644 index 00000000000..a1b050796bc --- /dev/null +++ b/solr/licenses/curator-client-5.7.1.jar.sha1 @@ -0,0 +1 @@ +a3637b9c1cdc3360e9f266ccaad2ad46f06e1ebb diff --git a/solr/licenses/curator-framework-5.7.0.jar.sha1 b/solr/licenses/curator-framework-5.7.0.jar.sha1 deleted file mode 100644 index c826bcd8a3b..00000000000 --- a/solr/licenses/curator-framework-5.7.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0b3a2c5191cb2aa609cc9bdd19701bdbd53ee627 diff --git a/solr/licenses/curator-framework-5.7.1.jar.sha1 b/solr/licenses/curator-framework-5.7.1.jar.sha1 new file mode 100644 index 00000000000..a64e2cb9b2b --- /dev/null +++ b/solr/licenses/curator-framework-5.7.1.jar.sha1 @@ -0,0 +1 @@ +ce5c5be53ec41c9f0a1f5a970e87cdae0f566aa3 diff --git a/solr/licenses/curator-framework-LICENSE-ASL.txt b/solr/licenses/curator-framework-LICENSE-ASL.txt deleted file mode 100644 index 7a4a3ea2424..00000000000 --- a/solr/licenses/curator-framework-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. \ No newline at end of file diff --git a/solr/licenses/curator-framework-NOTICE.txt b/solr/licenses/curator-framework-NOTICE.txt deleted file mode 100644 index f568d0fe32e..00000000000 --- a/solr/licenses/curator-framework-NOTICE.txt +++ /dev/null @@ -1,5 +0,0 @@ -Apache Curator -Copyright 2013-2014 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/curator-recipes-5.7.0.jar.sha1 b/solr/licenses/curator-recipes-5.7.0.jar.sha1 deleted file mode 100644 index 3b84a54317c..00000000000 --- a/solr/licenses/curator-recipes-5.7.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6cd1b05889cc4ab8f074366dbd0493c6d3b87b6f diff --git a/solr/licenses/curator-recipes-5.7.1.jar.sha1 b/solr/licenses/curator-recipes-5.7.1.jar.sha1 new file mode 100644 index 00000000000..fc1cd77aa2c --- /dev/null +++ b/solr/licenses/curator-recipes-5.7.1.jar.sha1 @@ -0,0 +1 @@ +39f8871614f0ed157f87f412bed8fd7b32c457ac diff --git a/solr/licenses/curator-recipes-NOTICE.txt b/solr/licenses/curator-recipes-NOTICE.txt deleted file mode 100644 index f568d0fe32e..00000000000 --- a/solr/licenses/curator-recipes-NOTICE.txt +++ /dev/null @@ -1,5 +0,0 @@ -Apache Curator -Copyright 2013-2014 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/curator-test-5.7.1.jar.sha1 b/solr/licenses/curator-test-5.7.1.jar.sha1 new file mode 100644 index 00000000000..7c40b3c04f2 --- /dev/null +++ b/solr/licenses/curator-test-5.7.1.jar.sha1 @@ -0,0 +1 @@ +743367fe601ba7ae481a1fa610b78790eeec1f7c diff --git a/solr/licenses/junit-jupiter-api-5.6.2.jar.sha1 b/solr/licenses/junit-jupiter-api-5.6.2.jar.sha1 new file mode 100644 index 00000000000..4c77692c079 --- /dev/null +++ b/solr/licenses/junit-jupiter-api-5.6.2.jar.sha1 @@ -0,0 +1 @@ +c9ba885abfe975cda123bf6f8f0a69a1b46956d0 diff --git a/solr/licenses/junit-platform-commons-1.6.2.jar.sha1 b/solr/licenses/junit-platform-commons-1.6.2.jar.sha1 new file mode 100644 index 00000000000..bbd0e93baf5 --- /dev/null +++ b/solr/licenses/junit-platform-commons-1.6.2.jar.sha1 @@ -0,0 +1 @@ +7644a14b329e76b5fe487628b50fb5eab6ba7d26 diff --git a/solr/licenses/opentest4j-1.2.0.jar.sha1 b/solr/licenses/opentest4j-1.2.0.jar.sha1 new file mode 100644 index 00000000000..1d8cede8888 --- /dev/null +++ b/solr/licenses/opentest4j-1.2.0.jar.sha1 @@ -0,0 +1 @@ +28c11eb91f9b6d8e200631d46e20a7f407f2a046 diff --git a/solr/licenses/curator-recipes-LICENSE-ASL.txt b/solr/licenses/opentest4j-LICENSE-ASL.txt similarity index 99% rename from solr/licenses/curator-recipes-LICENSE-ASL.txt rename to solr/licenses/opentest4j-LICENSE-ASL.txt index 7a4a3ea2424..8f71f43fee3 100644 --- a/solr/licenses/curator-recipes-LICENSE-ASL.txt +++ b/solr/licenses/opentest4j-LICENSE-ASL.txt @@ -1,4 +1,3 @@ - Apache License Version 2.0, January 2004 http://www.apache.org/licenses/ @@ -179,7 +178,7 @@ APPENDIX: How to apply the Apache License to your work. To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" + boilerplate notice, with the fields enclosed by brackets "{}" replaced with your own identifying information. (Don't include the brackets!) The text should be enclosed in the appropriate comment syntax for the file format. We also recommend that a @@ -187,7 +186,7 @@ same "printed page" as the copyright notice for easier identification within third-party archives. - Copyright [yyyy] [name of copyright owner] + Copyright {yyyy} {name of copyright owner} Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -199,4 +198,5 @@ 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. \ No newline at end of file + limitations under the License. + diff --git a/solr/licenses/opentest4j-NOTICE.txt b/solr/licenses/opentest4j-NOTICE.txt new file mode 100644 index 00000000000..5a42e15815d --- /dev/null +++ b/solr/licenses/opentest4j-NOTICE.txt @@ -0,0 +1,13 @@ +Copyright 2015-2023 the original author or authors. + +Licensed 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. diff --git a/solr/modules/hadoop-auth/build.gradle b/solr/modules/hadoop-auth/build.gradle index 6fc729461c7..a038264f1f7 100644 --- a/solr/modules/hadoop-auth/build.gradle +++ b/solr/modules/hadoop-auth/build.gradle @@ -50,14 +50,15 @@ dependencies { implementation('org.apache.zookeeper:zookeeper', { exclude group: "org.apache.yetus", module: "audience-annotations" }) - implementation ('org.apache.zookeeper:zookeeper-jute') { - exclude group: 'org.apache.yetus', module: 'audience-annotations' - } // required for instantiating a Zookeeper server (for embedding ZK or running tests) runtimeOnly 'org.xerial.snappy:snappy-java' - implementation 'org.apache.curator:curator-client' - implementation 'org.apache.curator:curator-framework' - runtimeOnly 'org.apache.curator:curator-recipes' + + implementation('org.apache.curator:curator-framework', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) + runtimeOnly('org.apache.curator:curator-recipes', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) // Hadoop auth framework implementation 'org.apache.hadoop:hadoop-annotations' @@ -104,9 +105,15 @@ dependencies { }) // Zookeeper dependency - some tests like HdfsCloudBackupRestore need this + testImplementation('org.apache.curator:curator-client', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) testImplementation('org.apache.zookeeper:zookeeper', { exclude group: "org.apache.yetus", module: "audience-annotations" }) + testImplementation ('org.apache.zookeeper:zookeeper-jute') { + exclude group: 'org.apache.yetus', module: 'audience-annotations' + } // required for instantiating a Zookeeper server in tests or embedded testRuntimeOnly ('org.xerial.snappy:snappy-java') } diff --git a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/DelegationTokenKerberosFilter.java b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/DelegationTokenKerberosFilter.java index b0021c889a5..318ad72e7e7 100644 --- a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/DelegationTokenKerberosFilter.java +++ b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/DelegationTokenKerberosFilter.java @@ -20,23 +20,15 @@ import static org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH_DEAFULT; import java.io.IOException; -import java.util.ArrayList; import java.util.Enumeration; -import java.util.List; import java.util.Objects; -import java.util.concurrent.ExecutorService; import javax.servlet.FilterChain; import javax.servlet.FilterConfig; import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; -import org.apache.curator.RetryPolicy; -import org.apache.curator.framework.AuthInfo; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.api.ACLProvider; -import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.server.AuthenticationHandler; @@ -44,13 +36,8 @@ import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation; import org.apache.solr.common.cloud.SecurityAwareZkACLProvider; import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.common.cloud.ZkACLProvider; -import org.apache.solr.common.cloud.ZkCredentialsProvider; -import org.apache.solr.common.util.ExecutorUtil; -import org.apache.solr.common.util.SolrNamedThreadFactory; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.data.ACL; /** * This is an authentication filter based on Hadoop's {@link DelegationTokenAuthenticationFilter}. @@ -58,9 +45,6 @@ * reuse the authentication of an end-user or another application. */ public class DelegationTokenKerberosFilter extends DelegationTokenAuthenticationFilter { - private ExecutorService curatorSafeServiceExecutor; - private CuratorFramework curatorFramework; - @Override public void init(FilterConfig conf) throws ServletException { if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider"))) { @@ -129,19 +113,6 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo super.doFilter(request, response, filterChainWrapper); } - @Override - public void destroy() { - super.destroy(); - if (curatorFramework != null) { - curatorFramework.close(); - curatorFramework = null; - } - if (curatorSafeServiceExecutor != null) { - ExecutorUtil.shutdownNowAndAwaitTermination(curatorSafeServiceExecutor); - curatorSafeServiceExecutor = null; - } - } - @Override protected void initializeAuthHandler(String authHandlerClassName, FilterConfig filterConfig) throws ServletException { @@ -183,108 +154,20 @@ private CuratorFramework getCuratorClientInternal(FilterConfig conf, SolrZkClien protected CuratorFramework getCuratorClient(SolrZkClient zkClient) throws InterruptedException, KeeperException { - // should we try to build a RetryPolicy off of the ZkController? - RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); - if (zkClient == null) { - throw new IllegalArgumentException("zkClient required"); - } - String zkHost = zkClient.getZkServerAddress(); - String zkChroot = zkHost.contains("/") ? zkHost.substring(zkHost.indexOf('/')) : ""; - String zkNamespace = zkChroot + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH; - zkNamespace = zkNamespace.startsWith("/") ? zkNamespace.substring(1) : zkNamespace; - String zkConnectionString = - zkHost.contains("/") ? zkHost.substring(0, zkHost.indexOf('/')) : zkHost; - SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient); - final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561. - // Create /security znode upfront. Without this, the curator framework creates this directory // path // without the appropriate ACL configuration. This issue is possibly related to HADOOP-11973 try { zkClient.makePath( SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, CreateMode.PERSISTENT, true); - } catch (KeeperException.NodeExistsException ignore) { + } catch (KeeperException.NodeExistsException ex) { + // ignore? } - curatorSafeServiceExecutor = - ExecutorUtil.newMDCAwareSingleThreadExecutor( - new SolrNamedThreadFactory("delegationtokenkerberosfilter-curator-safeService")); - curatorFramework = - CuratorFrameworkFactory.builder() - .namespace(zkNamespace) - .connectString(zkConnectionString) - .retryPolicy(retryPolicy) - .aclProvider(curatorToSolrZk.getACLProvider()) - .authorization(curatorToSolrZk.getAuthInfos()) - .sessionTimeoutMs(zkClient.getZkClientTimeout()) - .connectionTimeoutMs(connectionTimeoutMs) - .runSafeService(curatorSafeServiceExecutor) - .build(); - curatorFramework.start(); - - return curatorFramework; - } - - /** Convert Solr Zk Credentials/ACLs to Curator versions */ - protected static class SolrZkToCuratorCredentialsACLs { - private final String zkChroot; - private final ACLProvider aclProvider; - private final List authInfos; - - public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) { - this.aclProvider = createACLProvider(zkClient); - this.authInfos = createAuthInfo(zkClient); - String zkHost = zkClient.getZkServerAddress(); - this.zkChroot = zkHost.contains("/") ? zkHost.substring(zkHost.indexOf('/')) : null; - } + // Note - Curator complains if the namespace starts with / + String namespace = zkClient.getAbsolutePath(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH); + namespace = namespace.startsWith("/") ? namespace.substring(1) : namespace; - public ACLProvider getACLProvider() { - return aclProvider; - } - - public List getAuthInfos() { - return authInfos; - } - - private ACLProvider createACLProvider(SolrZkClient zkClient) { - final ZkACLProvider zkACLProvider = zkClient.getZkACLProvider(); - return new ACLProvider() { - @Override - public List getDefaultAcl() { - return zkACLProvider.getACLsToAdd(null); - } - - @Override - public List getAclForPath(String path) { - List acls = null; - - // The logic in SecurityAwareZkACLProvider does not work when - // the Solr zkPath is chrooted (e.g. /solr instead of /). This - // due to the fact that the getACLsToAdd(..) callback provides - // an absolute path (instead of relative path to the chroot) and - // the string comparison in SecurityAwareZkACLProvider fails. - if (zkACLProvider instanceof SecurityAwareZkACLProvider && zkChroot != null) { - acls = zkACLProvider.getACLsToAdd(path.replace(zkChroot, "")); - } else { - acls = zkACLProvider.getACLsToAdd(path); - } - - return acls; - } - }; - } - - private List createAuthInfo(SolrZkClient zkClient) { - List ret = new ArrayList<>(); - - // In theory the credentials to add could change here if zookeeper hasn't been initialized - ZkCredentialsProvider credentialsProvider = - zkClient.getZkClientConnectionStrategy().getZkCredentialsToAddAutomatically(); - for (ZkCredentialsProvider.ZkCredentials zkCredentials : - credentialsProvider.getCredentials()) { - ret.add(new AuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth())); - } - return ret; - } + return zkClient.getCuratorFramework().usingNamespace(namespace); } } diff --git a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthFilter.java b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthFilter.java index f462ad7e682..25bd363d3ad 100644 --- a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthFilter.java +++ b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthFilter.java @@ -20,35 +20,22 @@ import static org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH_DEAFULT; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.Objects; -import java.util.concurrent.ExecutorService; import javax.servlet.FilterChain; import javax.servlet.FilterConfig; import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; -import org.apache.curator.RetryPolicy; -import org.apache.curator.framework.AuthInfo; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.api.ACLProvider; -import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.server.AuthenticationHandler; import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter; import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation; import org.apache.solr.common.cloud.SecurityAwareZkACLProvider; import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.common.cloud.ZkACLProvider; -import org.apache.solr.common.cloud.ZkCredentialsProvider; -import org.apache.solr.common.util.ExecutorUtil; -import org.apache.solr.common.util.SolrNamedThreadFactory; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.data.ACL; /** * This is an authentication filter based on Hadoop's {@link DelegationTokenAuthenticationFilter}. @@ -60,9 +47,6 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter { */ static final String DELEGATION_TOKEN_ZK_CLIENT = "solr.kerberos.delegation.token.zk.client"; - private ExecutorService curatorSafeServiceExecutor; - private CuratorFramework curatorFramework; - @Override public void init(FilterConfig conf) throws ServletException { if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider"))) { @@ -111,14 +95,6 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo @Override public void destroy() { super.destroy(); - if (curatorFramework != null) { - curatorFramework.close(); - curatorFramework = null; - } - if (curatorSafeServiceExecutor != null) { - ExecutorUtil.shutdownNowAndAwaitTermination(curatorSafeServiceExecutor); - curatorSafeServiceExecutor = null; - } } @Override @@ -161,109 +137,21 @@ private CuratorFramework getCuratorClientInternal(FilterConfig conf, SolrZkClien } protected CuratorFramework getCuratorClient(SolrZkClient zkClient) - throws KeeperException, InterruptedException { - // should we try to build a RetryPolicy off of the ZkController? - RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); - if (zkClient == null) { - throw new IllegalArgumentException("zkClient required"); - } - String zkHost = zkClient.getZkServerAddress(); - String zkChroot = zkHost.contains("/") ? zkHost.substring(zkHost.indexOf('/')) : ""; - String zkNamespace = zkChroot + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH; - zkNamespace = zkNamespace.startsWith("/") ? zkNamespace.substring(1) : zkNamespace; - String zkConnectionString = - zkHost.contains("/") ? zkHost.substring(0, zkHost.indexOf('/')) : zkHost; - SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient); - final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561. - + throws InterruptedException, KeeperException { // Create /security znode upfront. Without this, the curator framework creates this directory // path // without the appropriate ACL configuration. This issue is possibly related to HADOOP-11973 try { zkClient.makePath( SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, CreateMode.PERSISTENT, true); - } catch (KeeperException.NodeExistsException ignore) { + } catch (KeeperException.NodeExistsException ex) { + // ignore? } - curatorSafeServiceExecutor = - ExecutorUtil.newMDCAwareSingleThreadExecutor( - new SolrNamedThreadFactory("hadoopauthfilter-curator-safeService")); - curatorFramework = - CuratorFrameworkFactory.builder() - .namespace(zkNamespace) - .connectString(zkConnectionString) - .retryPolicy(retryPolicy) - .aclProvider(curatorToSolrZk.getACLProvider()) - .authorization(curatorToSolrZk.getAuthInfos()) - .sessionTimeoutMs(zkClient.getZkClientTimeout()) - .connectionTimeoutMs(connectionTimeoutMs) - .runSafeService(curatorSafeServiceExecutor) - .build(); - curatorFramework.start(); + // Note - Curator complains if the namespace starts with / + String namespace = zkClient.getAbsolutePath(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH); + namespace = namespace.startsWith("/") ? namespace.substring(1) : namespace; - return curatorFramework; - } - - /** Convert Solr Zk Credentials/ACLs to Curator versions */ - protected static class SolrZkToCuratorCredentialsACLs { - private final String zkChroot; - private final ACLProvider aclProvider; - private final List authInfos; - - public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) { - this.aclProvider = createACLProvider(zkClient); - this.authInfos = createAuthInfo(zkClient); - String zkHost = zkClient.getZkServerAddress(); - this.zkChroot = zkHost.contains("/") ? zkHost.substring(zkHost.indexOf('/')) : null; - } - - public ACLProvider getACLProvider() { - return aclProvider; - } - - public List getAuthInfos() { - return authInfos; - } - - private ACLProvider createACLProvider(SolrZkClient zkClient) { - final ZkACLProvider zkACLProvider = zkClient.getZkACLProvider(); - return new ACLProvider() { - @Override - public List getDefaultAcl() { - return zkACLProvider.getACLsToAdd(null); - } - - @Override - public List getAclForPath(String path) { - List acls = null; - - // The logic in SecurityAwareZkACLProvider does not work when - // the Solr zkPath is chrooted (e.g. /solr instead of /). This - // due to the fact that the getACLsToAdd(..) callback provides - // an absolute path (instead of relative path to the chroot) and - // the string comparison in SecurityAwareZkACLProvider fails. - if (zkACLProvider instanceof SecurityAwareZkACLProvider && zkChroot != null) { - acls = zkACLProvider.getACLsToAdd(path.replace(zkChroot, "")); - } else { - acls = zkACLProvider.getACLsToAdd(path); - } - - return acls; - } - }; - } - - private List createAuthInfo(SolrZkClient zkClient) { - List ret = new ArrayList<>(); - - // In theory the credentials to add could change here if zookeeper hasn't been initialized - ZkCredentialsProvider credentialsProvider = - zkClient.getZkClientConnectionStrategy().getZkCredentialsToAddAutomatically(); - for (ZkCredentialsProvider.ZkCredentials zkCredentials : - credentialsProvider.getCredentials()) { - ret.add(new AuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth())); - } - return ret; - } + return zkClient.getCuratorFramework().usingNamespace(namespace); } } diff --git a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthPlugin.java b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthPlugin.java index f878ef49b2c..b103ee6919e 100644 --- a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthPlugin.java +++ b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/HadoopAuthPlugin.java @@ -35,6 +35,7 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.security.authentication.server.AuthenticationFilter; +import org.apache.hadoop.security.authentication.util.ZKSignerSecretProvider; import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler; import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder; import org.apache.solr.cloud.ZkController; @@ -190,6 +191,7 @@ protected FilterConfig getInitFilterConfig(Map pluginConfig) { ZkController controller = coreContainer.getZkController(); if (controller != null) { servletContext.setAttribute(DELEGATION_TOKEN_ZK_CLIENT, controller.getZkClient()); + params.put(ZKSignerSecretProvider.DISCONNECT_FROM_ZOOKEEPER_ON_SHUTDOWN, "false"); } FilterConfig conf = diff --git a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/KerberosPlugin.java b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/KerberosPlugin.java index 673e485667a..66b9b90dee2 100644 --- a/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/KerberosPlugin.java +++ b/solr/modules/hadoop-auth/src/java/org/apache/solr/security/hadoop/KerberosPlugin.java @@ -32,6 +32,7 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.security.authentication.util.KerberosName; +import org.apache.hadoop.security.authentication.util.ZKSignerSecretProvider; import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler; import org.apache.http.HttpRequest; import org.apache.http.protocol.HttpContext; @@ -158,6 +159,8 @@ protected FilterConfig getInitFilterConfig( "/token"); // ensure krb5 is setup properly before running curator getHttpClientBuilder(SolrHttpClientBuilder.create()); + + params.put(ZKSignerSecretProvider.DISCONNECT_FROM_ZOOKEEPER_ON_SHUTDOWN, "false"); } } else { log.info( diff --git a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/SaslZkACLProviderTest.java b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/SaslZkACLProviderTest.java index fc39eb0aad8..b3601b5e7fd 100644 --- a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/SaslZkACLProviderTest.java +++ b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/SaslZkACLProviderTest.java @@ -24,6 +24,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.util.concurrent.TimeUnit; +import org.apache.curator.framework.api.ACLProvider; import org.apache.lucene.tests.util.QuickPatchThreadsFilter; import org.apache.lucene.util.Constants; import org.apache.solr.SolrIgnoredThreadsFilter; @@ -94,7 +95,8 @@ public void setUp() throws Exception { try (SolrZkClient zkClient = new SolrZkClientWithACLs(zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT)) { - ZooKeeperSaslClient saslClient = zkClient.getZooKeeper().getSaslClient(); + ZooKeeperSaslClient saslClient = + zkClient.getCuratorFramework().getZookeeperClient().getZooKeeper().getSaslClient(); assumeFalse("Could not set up ZK with SASL", saslClient.isFailed()); zkClient.makePath("/solr", false, true); } catch (KeeperException e) { @@ -188,7 +190,7 @@ public SolrZkClientWithACLs(String zkServerAddress, int zkClientTimeout) { } @Override - public ZkACLProvider createZkACLProvider() { + public ZkACLProvider createACLProvider() { return new SaslZkACLProvider(); } } @@ -204,7 +206,7 @@ public SolrZkClientNoACLs(String zkServerAddress, int zkClientTimeout) { } @Override - public ZkACLProvider createZkACLProvider() { + public ACLProvider createACLProvider() { return new DefaultZkACLProvider(); } } diff --git a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java index 8679b2788ea..c940391b01e 100644 --- a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java +++ b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java @@ -53,8 +53,7 @@ public static void setupClass() throws Exception { InetAddress loopback = InetAddress.getLoopbackAddress(); Path securityJsonPath = TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegation.json"); - String securityJson = - new String(Files.readAllBytes(securityJsonPath), Charset.defaultCharset()); + String securityJson = Files.readString(securityJsonPath, Charset.defaultCharset()); Map securityConfig = (Map) Utils.fromJSONString(securityJson); Map authConfig = (Map) securityConfig.get("authentication"); diff --git a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithDelegationTokens.java b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithDelegationTokens.java index dfd2d58ed5b..7f2528aa7ca 100644 --- a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithDelegationTokens.java +++ b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithDelegationTokens.java @@ -78,10 +78,6 @@ public static void startup() throws Exception { @AfterClass public static void shutdown() throws Exception { - if (miniCluster != null) { - miniCluster.shutdown(); - miniCluster = null; - } if (null != solrClientPrimary) { solrClientPrimary.close(); solrClientPrimary = null; @@ -90,6 +86,11 @@ public static void shutdown() throws Exception { solrClientSecondary.close(); solrClientSecondary = null; } + + if (miniCluster != null) { + miniCluster.shutdown(); + miniCluster = null; + } System.clearProperty("authenticationPlugin"); System.clearProperty(KerberosPlugin.DELEGATION_TOKEN_ENABLED); System.clearProperty("solr.kerberos.cookie.domain"); diff --git a/solr/solrj-zookeeper/build.gradle b/solr/solrj-zookeeper/build.gradle index 2e03d9fed3c..5c63a90e05c 100644 --- a/solr/solrj-zookeeper/build.gradle +++ b/solr/solrj-zookeeper/build.gradle @@ -35,6 +35,13 @@ dependencies { // declare dependencies we use even though already declared by solrj-core implementation 'org.slf4j:slf4j-api' + + implementation('org.apache.curator:curator-client', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) + api('org.apache.curator:curator-framework', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) api('org.apache.zookeeper:zookeeper', { exclude group: "org.apache.yetus", module: "audience-annotations" }) diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/cloud/DistribStateManager.java b/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/cloud/DistribStateManager.java index a8ca13e8be7..4c9ed44cf32 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/cloud/DistribStateManager.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/cloud/DistribStateManager.java @@ -24,13 +24,13 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import org.apache.curator.framework.api.transaction.CuratorTransactionResult; import org.apache.solr.common.SolrCloseable; import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.PerReplicaStates; +import org.apache.solr.common.cloud.SolrZkClient; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Op; -import org.apache.zookeeper.OpResult; import org.apache.zookeeper.Watcher; /** This interface represents a distributed state repository. */ @@ -92,7 +92,7 @@ void setData(String path, byte[] data, int version) KeeperException, InterruptedException; - List multi(final Iterable ops) + List multi(final List ops) throws BadVersionException, NoSuchElementException, AlreadyExistsException, diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java b/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java index 815a5bd7b4f..a71542cfbdc 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import org.apache.curator.framework.api.transaction.CuratorTransactionResult; import org.apache.solr.client.solrj.cloud.AlreadyExistsException; import org.apache.solr.client.solrj.cloud.BadVersionException; import org.apache.solr.client.solrj.cloud.DistribStateManager; @@ -34,8 +35,6 @@ import org.apache.solr.common.util.Utils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Op; -import org.apache.zookeeper.OpResult; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.Stat; @@ -199,7 +198,7 @@ public void setData(String path, byte[] data, int version) } @Override - public List multi(Iterable ops) + public List multi(List ops) throws BadVersionException, AlreadyExistsException, NoSuchElementException, @@ -207,7 +206,7 @@ public List multi(Iterable ops) KeeperException, InterruptedException { try { - return zkClient.multi(ops, true); + return zkClient.multi(ops); } catch (KeeperException.NoNodeException e) { throw new NoSuchElementException(ops.toString()); } catch (KeeperException.NodeExistsException e) { diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/BeforeReconnect.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/BeforeReconnect.java index 928e606c920..6e028056013 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/BeforeReconnect.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/BeforeReconnect.java @@ -16,6 +16,17 @@ */ package org.apache.solr.common.cloud; -public interface BeforeReconnect { +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.state.ConnectionState; +import org.apache.curator.framework.state.ConnectionStateListener; + +public interface BeforeReconnect extends ConnectionStateListener { public void command(); + + @Override + default void stateChanged(CuratorFramework client, ConnectionState newState) { + if (newState == ConnectionState.LOST || newState == ConnectionState.SUSPENDED) { + command(); + } + } } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ConnectionManager.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ConnectionManager.java deleted file mode 100644 index ba527b1eef8..00000000000 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ConnectionManager.java +++ /dev/null @@ -1,318 +0,0 @@ -/* - * 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 org.apache.zookeeper.Watcher.Event.KeeperState.AuthFailed; -import static org.apache.zookeeper.Watcher.Event.KeeperState.Disconnected; -import static org.apache.zookeeper.Watcher.Event.KeeperState.Expired; - -import java.lang.invoke.MethodHandles; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import org.apache.solr.common.SolrException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.Watcher.Event.KeeperState; -import org.apache.zookeeper.ZooKeeper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ConnectionManager implements Watcher { - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - private final String name; - - private volatile boolean connected = false; - - private final ZkClientConnectionStrategy connectionStrategy; - - private final String zkServerAddress; - - private final SolrZkClient client; - - private final OnReconnect onReconnect; - private final BeforeReconnect beforeReconnect; - - private volatile boolean isClosed = false; - - // Track the likely expired state - private static class LikelyExpiredState { - private static LikelyExpiredState NOT_EXPIRED = - new LikelyExpiredState(StateType.NOT_EXPIRED, 0); - private static LikelyExpiredState EXPIRED = new LikelyExpiredState(StateType.EXPIRED, 0); - - public enum StateType { - NOT_EXPIRED, // definitely not expired - EXPIRED, // definitely expired - TRACKING_TIME // not sure, tracking time of last disconnect - } - - private StateType stateType; - private long lastDisconnectTime; - - public LikelyExpiredState(StateType stateType, long lastDisconnectTime) { - this.stateType = stateType; - this.lastDisconnectTime = lastDisconnectTime; - } - - public boolean isLikelyExpired(long timeToExpire) { - return stateType == StateType.EXPIRED - || (stateType == StateType.TRACKING_TIME - && (System.nanoTime() - lastDisconnectTime - > TimeUnit.NANOSECONDS.convert(timeToExpire, TimeUnit.MILLISECONDS))); - } - } - - public interface IsClosed { - boolean isClosed(); - } - - private volatile LikelyExpiredState likelyExpiredState = LikelyExpiredState.EXPIRED; - - private IsClosed isClosedCheck; - - public ConnectionManager( - String name, - SolrZkClient client, - String zkServerAddress, - ZkClientConnectionStrategy strat, - OnReconnect onConnect, - BeforeReconnect beforeReconnect, - IsClosed isClosed) { - this.name = name; - this.client = client; - this.connectionStrategy = strat; - this.zkServerAddress = zkServerAddress; - this.onReconnect = onConnect; - this.beforeReconnect = beforeReconnect; - this.isClosedCheck = isClosed; - } - - private synchronized void connected() { - connected = true; - likelyExpiredState = LikelyExpiredState.NOT_EXPIRED; - notifyAll(); - } - - private synchronized void disconnected() { - connected = false; - // record the time we expired unless we are already likely expired - if (!likelyExpiredState.isLikelyExpired(0)) { - likelyExpiredState = - new LikelyExpiredState(LikelyExpiredState.StateType.TRACKING_TIME, System.nanoTime()); - } - notifyAll(); - } - - @Override - public void process(WatchedEvent event) { - if (event.getState() == AuthFailed - || event.getState() == Disconnected - || event.getState() == Expired) { - log.warn( - "Watcher {} name: {} got event {} path: {} type: {}", - this, - name, - event, - event.getPath(), - event.getType()); - } else { - if (log.isDebugEnabled()) { - log.debug( - "Watcher {} name: {} got event {} path: {} type: {}", - this, - name, - event, - event.getPath(), - event.getType()); - } - } - - if (isClosed()) { - log.debug("Client->ZooKeeper status change trigger but we are already closed"); - return; - } - - KeeperState state = event.getState(); - - if (state == KeeperState.SyncConnected) { - log.info("zkClient has connected"); - connected(); - connectionStrategy.connected(); - } else if (state == Expired) { - if (isClosed()) { - return; - } - // we don't call disconnected here, because we know we are expired - connected = false; - likelyExpiredState = LikelyExpiredState.EXPIRED; - - log.warn( - "Our previous ZooKeeper session was expired. Attempting to reconnect to recover relationship with ZooKeeper..."); - - if (beforeReconnect != null) { - try { - beforeReconnect.command(); - } catch (Exception e) { - log.warn("Exception running beforeReconnect command", e); - } - } - - do { - // This loop will break if a valid connection is made. If a connection is not made then it - // will repeat and try again to create a new connection. - try { - connectionStrategy.reconnect( - zkServerAddress, - client.getZkClientTimeout(), - this, - new ZkClientConnectionStrategy.ZkUpdate() { - @Override - public void update(ZooKeeper keeper) { - try { - waitForConnected(Long.MAX_VALUE); - - try { - client.updateKeeper(keeper); - } catch (InterruptedException e) { - closeKeeper(keeper); - Thread.currentThread().interrupt(); - // we must have been asked to stop - throw new RuntimeException(e); - } - - if (onReconnect != null) { - onReconnect.command(); - } - - } catch (Exception e1) { - // if there was a problem creating the new SolrZooKeeper - // or if we cannot run our reconnect command, close the keeper - // our retry loop will try to create one again - closeKeeper(keeper); - throw new RuntimeException(e1); - } - } - - private void closeKeeper(ZooKeeper keeper) { - try { - keeper.close(); - } catch (InterruptedException e) { - // Restore the interrupted status - Thread.currentThread().interrupt(); - log.error("", e); - throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e); - } - } - }); - - break; - - } catch (Exception e) { - log.error("Could not connect due to error, sleeping for 1s and trying again", e); - waitSleep(1000); - } - - } while (!isClosed()); - log.info("zkClient Connected: {}", connected); - } else if (state == KeeperState.Disconnected) { - log.warn("zkClient has disconnected"); - disconnected(); - connectionStrategy.disconnected(); - } else if (state == KeeperState.AuthFailed) { - log.warn("zkClient received AuthFailed"); - } - } - - public synchronized boolean isConnectedAndNotClosed() { - return !isClosed() && connected; - } - - public synchronized boolean isConnected() { - return connected; - } - - // we use a volatile rather than sync - // to avoid possible deadlock on shutdown - public void close() { - this.isClosed = true; - this.likelyExpiredState = LikelyExpiredState.EXPIRED; - } - - private boolean isClosed() { - return isClosed || isClosedCheck.isClosed(); - } - - public boolean isLikelyExpired() { - return isClosed() - || likelyExpiredState.isLikelyExpired((long) (client.getZkClientTimeout() * 0.90)); - } - - public synchronized void waitSleep(long waitFor) { - try { - wait(waitFor); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - - /** - * Wait for an established zookeeper connection - * - * @param waitForConnection time to wait, in ms - */ - public synchronized void waitForConnected(long waitForConnection) throws TimeoutException { - log.info("Waiting up to {}ms for client to connect to ZooKeeper", waitForConnection); - long expire = - System.nanoTime() + TimeUnit.NANOSECONDS.convert(waitForConnection, TimeUnit.MILLISECONDS); - long left = 1; - while (!connected && left > 0) { - if (isClosed()) { - break; - } - try { - wait(500); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; - } - left = expire - System.nanoTime(); - } - if (!connected) { - throw new TimeoutException( - "Could not connect to ZooKeeper " - + zkServerAddress - + " within " - + waitForConnection - + " ms"); - } - log.info("Client is connected to ZooKeeper"); - } - - public synchronized void waitForDisconnected(long timeout) - throws InterruptedException, TimeoutException { - long expire = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS); - long left = timeout; - while (connected && left > 0) { - wait(left); - left = expire - System.nanoTime(); - } - if (connected) { - throw new TimeoutException("Did not disconnect"); - } - } -} diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java deleted file mode 100644 index 42d8cbe31e9..00000000000 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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 java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.concurrent.TimeoutException; -import org.apache.solr.common.AlreadyClosedException; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** TODO: improve backoff retry impl */ -public class DefaultConnectionStrategy extends ZkClientConnectionStrategy { - - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @Override - public void connect(String serverAddress, int timeout, Watcher watcher, ZkUpdate updater) - throws IOException, InterruptedException, TimeoutException { - ZooKeeper zk = createZooKeeper(serverAddress, timeout, watcher); - boolean success = false; - try { - updater.update(zk); - success = true; - } finally { - if (!success) { - zk.close(); - } - } - } - - @Override - public void reconnect( - final String serverAddress, - final int zkClientTimeout, - final Watcher watcher, - final ZkUpdate updater) - throws IOException, InterruptedException, TimeoutException { - log.warn("Connection expired - starting a new one..."); - ZooKeeper zk = createZooKeeper(serverAddress, zkClientTimeout, watcher); - boolean success = false; - try { - updater.update(zk); - success = true; - log.info("Reconnected to ZooKeeper"); - } catch (AlreadyClosedException e) { - - } finally { - if (!success) { - try { - zk.close(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - } -} diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkACLProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkACLProvider.java index 5b43414da9d..2582d01a0b9 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkACLProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkACLProvider.java @@ -22,7 +22,15 @@ public class DefaultZkACLProvider implements ZkACLProvider { - private List globalACLsToAdd; + private volatile List globalACLsToAdd; + + public DefaultZkACLProvider() { + this(null); + } + + public DefaultZkACLProvider(List globalACLsToAdd) { + this.globalACLsToAdd = globalACLsToAdd; + } @Override public List getACLsToAdd(String zNodePath) { @@ -35,6 +43,28 @@ public List getACLsToAdd(String zNodePath) { return globalACLsToAdd; } + @Override + public List getAclForPath(String zNodePath) { + // In default (simple) implementation use the same set of ACLs for all znodes + if (globalACLsToAdd == null) { + synchronized (this) { + if (globalACLsToAdd == null) globalACLsToAdd = createGlobalACLsToAdd(); + } + } + return globalACLsToAdd; + } + + @Override + public List getDefaultAcl() { + // In default (simple) implementation use the same set of ACLs for all znodes + if (globalACLsToAdd == null) { + synchronized (this) { + if (globalACLsToAdd == null) globalACLsToAdd = createGlobalACLsToAdd(); + } + } + return globalACLsToAdd; + } + protected List createGlobalACLsToAdd() { return ZooDefs.Ids.OPEN_ACL_UNSAFE; } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkCredentialsProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkCredentialsProvider.java index e855052f6a6..cfdd2098f94 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkCredentialsProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DefaultZkCredentialsProvider.java @@ -17,19 +17,30 @@ package org.apache.solr.common.cloud; import java.util.ArrayList; -import java.util.Collection; +import java.util.List; +import org.apache.curator.framework.AuthInfo; public class DefaultZkCredentialsProvider implements ZkCredentialsProvider { - private Collection zkCredentials; + private volatile List zkCredentials; protected ZkCredentialsInjector zkCredentialsInjector; public DefaultZkCredentialsProvider() { - this(new DefaultZkCredentialsInjector()); + this(new DefaultZkCredentialsInjector(), null); + } + + public DefaultZkCredentialsProvider(List zkCredentials) { + this(new DefaultZkCredentialsInjector(), zkCredentials); } public DefaultZkCredentialsProvider(ZkCredentialsInjector zkCredentialsInjector) { + this(zkCredentialsInjector, null); + } + + public DefaultZkCredentialsProvider( + ZkCredentialsInjector zkCredentialsInjector, List zkCredentials) { this.zkCredentialsInjector = zkCredentialsInjector; + this.zkCredentials = zkCredentials; } @Override @@ -38,7 +49,7 @@ public void setZkCredentialsInjector(ZkCredentialsInjector zkCredentialsInjector } @Override - public Collection getCredentials() { + public List getCredentials() { if (zkCredentials == null) { synchronized (this) { if (zkCredentials == null) zkCredentials = createCredentials(); @@ -47,7 +58,7 @@ public Collection getCredentials() { return zkCredentials; } - protected Collection createCredentials() { + protected List createCredentials() { return new ArrayList<>(); } } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DigestZkCredentialsProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DigestZkCredentialsProvider.java index cfe30286dc9..03cbbfd0c31 100755 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DigestZkCredentialsProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/DigestZkCredentialsProvider.java @@ -19,8 +19,8 @@ import java.lang.invoke.MethodHandles; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Collection; import java.util.List; +import org.apache.curator.framework.AuthInfo; import org.apache.solr.common.util.StrUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,9 +40,18 @@ public DigestZkCredentialsProvider(ZkCredentialsInjector zkCredentialsInjector) super(zkCredentialsInjector); } + public DigestZkCredentialsProvider(List zkCredentials) { + super(new DefaultZkCredentialsInjector(), zkCredentials); + } + + public DigestZkCredentialsProvider( + ZkCredentialsInjector zkCredentialsInjector, List zkCredentials) { + super(zkCredentialsInjector, zkCredentials); + } + @Override - protected Collection createCredentials() { - List result = new ArrayList<>(1); + protected List createCredentials() { + List result = new ArrayList<>(1); List zkCredentials = zkCredentialsInjector.getZkCredentials(); log.debug("createCredentials using zkCredentials: {}", zkCredentials); @@ -52,7 +61,7 @@ protected Collection createCredentials() { if (StrUtils.isNotNullOrEmpty(zkCredential.getUsername()) && StrUtils.isNotNullOrEmpty(zkCredential.getPassword())) { result.add( - new ZkCredentials( + new AuthInfo( "digest", (zkCredential.getUsername() + ":" + zkCredential.getPassword()) .getBytes(StandardCharsets.UTF_8))); diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/TestConnectionStrategy.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/OnDisconnect.java similarity index 54% rename from solr/test-framework/src/java/org/apache/solr/cloud/TestConnectionStrategy.java rename to solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/OnDisconnect.java index 88d94faaa85..f33013c5185 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/TestConnectionStrategy.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/OnDisconnect.java @@ -14,22 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.solr.cloud; +package org.apache.solr.common.cloud; -import java.io.IOException; -import org.apache.solr.common.cloud.DefaultConnectionStrategy; -import org.apache.zookeeper.TestableZooKeeper; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.state.ConnectionState; +import org.apache.curator.framework.state.ConnectionStateListener; + +public interface OnDisconnect extends ConnectionStateListener { + public void command(); -/** - * Connection strategy that creates instances of {@link TestableZooKeeper} instead of plain {@link - * ZooKeeper} objects. Useful for adding pause and disconnect events. - */ -public class TestConnectionStrategy extends DefaultConnectionStrategy { @Override - protected ZooKeeper newZooKeeperInstance( - String serverAddress, int zkClientTimeout, Watcher watcher) throws IOException { - return new TestableZooKeeper(serverAddress, zkClientTimeout, watcher); + default void stateChanged(CuratorFramework client, ConnectionState newState) { + if (newState == ConnectionState.LOST) { + command(); + } } } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/OnReconnect.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/OnReconnect.java index 19acfda4f6a..1863d797938 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/OnReconnect.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/OnReconnect.java @@ -16,7 +16,10 @@ */ package org.apache.solr.common.cloud; -import org.apache.zookeeper.KeeperException.SessionExpiredException; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.state.ConnectionState; +import org.apache.curator.framework.state.ConnectionStateListener; /** * Implementations are expected to implement a correct hashCode and equals method needed to uniquely @@ -25,6 +28,19 @@ * org.apache.solr.cloud.ZkController#removeOnReconnectListener(OnReconnect) when it no longer needs * to be notified of ZK reconnection events. */ -public interface OnReconnect { - void command() throws SessionExpiredException; +public interface OnReconnect extends ConnectionStateListener { + void command(); + + AtomicBoolean sessionEnded = new AtomicBoolean(false); + + @Override + default void stateChanged(CuratorFramework client, ConnectionState newState) { + if (ConnectionState.RECONNECTED.equals(newState)) { + if (sessionEnded.getAndSet(false)) { + command(); + } + } else if (ConnectionState.LOST == newState) { + sessionEnded.set(true); + } + } } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java index dd0b14fd3e1..85fcef00e83 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java @@ -30,7 +30,6 @@ import org.apache.solr.common.util.CommonTestInjection; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Op; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,18 +103,17 @@ private void persist( log.debug("Per-replica state being persisted for : '{}', ops: {}", znode, operations); } - List ops = new ArrayList<>(operations.size()); + List ops = new ArrayList<>(operations.size()); for (PerReplicaStates.Operation op : operations) { // the state of the replica is being updated String path = znode + "/" + op.state.asString; ops.add( op.typ == PerReplicaStates.Operation.Type.ADD - ? Op.create( - path, null, zkClient.getZkACLProvider().getACLsToAdd(path), CreateMode.PERSISTENT) - : Op.delete(path, -1)); + ? zkOp -> zkOp.create().withMode(CreateMode.PERSISTENT).forPath(path, null) + : zkOp -> zkOp.delete().withVersion(-1).forPath(path)); } try { - zkClient.multi(ops, true); + zkClient.multi(ops); } catch (KeeperException e) { log.error("Multi-op exception: {}", zkClient.getChildren(znode, null, true)); throw e; diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java index 2fbfbdd2ef8..ca2ba3bfe0a 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java @@ -17,7 +17,6 @@ package org.apache.solr.common.cloud; import java.util.Arrays; -import java.util.Collections; import java.util.List; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.data.ACL; @@ -43,6 +42,7 @@ protected List createNonSecurityACLsToAdd() { @Override protected List createSecurityACLsToAdd() { - return Collections.singletonList(new ACL(ZooDefs.Perms.ALL, new Id("sasl", superUser))); + // Must be Arrays.asList(), Zookeeper does not allow for immutable list types for ACLs + return Arrays.asList(new ACL(ZooDefs.Perms.ALL, new Id("sasl", superUser))); } } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java index 1868f53308a..e0a42c79ebb 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java @@ -17,6 +17,7 @@ package org.apache.solr.common.cloud; import java.util.List; +import org.apache.curator.utils.ZKPaths; import org.apache.zookeeper.data.ACL; /** @@ -26,16 +27,52 @@ public abstract class SecurityAwareZkACLProvider implements ZkACLProvider { public static final String SECURITY_ZNODE_PATH = "/security"; - private List nonSecurityACLsToAdd; - private List securityACLsToAdd; + private volatile List nonSecurityACLsToAdd; + private volatile List securityACLsToAdd; + + private final String securityConfPath; + private final String securityZNodePath; + private final String securityZNodePathDir; + protected ZkCredentialsInjector zkCredentialsInjector; public SecurityAwareZkACLProvider() { this(new DefaultZkCredentialsInjector()); } + public SecurityAwareZkACLProvider(String chroot) { + this(new DefaultZkCredentialsInjector(), chroot); + } + public SecurityAwareZkACLProvider(ZkCredentialsInjector zkCredentialsInjector) { + this(zkCredentialsInjector, null); + } + + public SecurityAwareZkACLProvider(ZkCredentialsInjector zkCredentialsInjector, String chroot) { this.zkCredentialsInjector = zkCredentialsInjector; + if (chroot != null) { + this.securityConfPath = ZKPaths.makePath(chroot, ZkStateReader.SOLR_SECURITY_CONF_PATH); + this.securityZNodePath = ZKPaths.makePath(chroot, SECURITY_ZNODE_PATH); + this.securityZNodePathDir = securityZNodePath + "/"; + } else { + this.securityConfPath = null; + this.securityZNodePath = null; + this.securityZNodePathDir = null; + } + } + + public SecurityAwareZkACLProvider withChroot(String chroot) { + return new SecurityAwareZkACLProvider(chroot) { + @Override + protected List createNonSecurityACLsToAdd() { + return SecurityAwareZkACLProvider.this.createNonSecurityACLsToAdd(); + } + + @Override + protected List createSecurityACLsToAdd() { + return SecurityAwareZkACLProvider.this.createSecurityACLsToAdd(); + } + }; } @Override @@ -52,11 +89,16 @@ public final List getACLsToAdd(String zNodePath) { } } + @Override + public final List getDefaultAcl() { + return getNonSecurityACLsToAdd(); + } + protected boolean isSecurityZNodePath(String zNodePath) { return zNodePath != null - && (zNodePath.equals(ZkStateReader.SOLR_SECURITY_CONF_PATH) - || zNodePath.equals(SECURITY_ZNODE_PATH) - || zNodePath.startsWith(SECURITY_ZNODE_PATH + "/")); + && (zNodePath.equals(securityConfPath) + || zNodePath.equals(securityZNodePath) + || (securityZNodePathDir != null && zNodePath.startsWith(securityZNodePathDir))); } /** diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java index 044500476de..9ae720cb180 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java @@ -23,24 +23,37 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.LongAdder; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Predicate; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.curator.RetryPolicy; +import org.apache.curator.ensemble.fixed.FixedEnsembleProvider; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.api.ACLProvider; +import org.apache.curator.framework.api.transaction.CuratorOp; +import org.apache.curator.framework.api.transaction.CuratorTransactionResult; +import org.apache.curator.framework.api.transaction.TransactionOp; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.ZKPaths; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.impl.SolrZkClientTimeout; import org.apache.solr.common.MapWriter; import org.apache.solr.common.SolrException; import org.apache.solr.common.annotation.JsonProperty; -import org.apache.solr.common.cloud.ConnectionManager.IsClosed; import org.apache.solr.common.util.Compressor; import org.apache.solr.common.util.ExecutorUtil; +import org.apache.solr.common.util.IOUtils; import org.apache.solr.common.util.ObjectReleaseTracker; import org.apache.solr.common.util.ReflectMapWriter; import org.apache.solr.common.util.SolrNamedThreadFactory; @@ -48,17 +61,12 @@ import org.apache.solr.common.util.ZLibCompressor; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NoAuthException; import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.KeeperException.NodeExistsException; -import org.apache.zookeeper.Op; -import org.apache.zookeeper.OpResult; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,11 +81,8 @@ public class SolrZkClient implements Closeable { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private ConnectionManager connManager; - - private volatile ZooKeeper keeper; - - private ZkCmdExecutor zkCmdExecutor; + private ExecutorService curatorSafeServiceExecutor; + private CuratorFramework client; private final ZkMetrics metrics = new ZkMetrics(); @@ -90,14 +95,13 @@ public MapWriter getMetrics() { private final ExecutorService zkCallbackExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("zkCallback")); - private final ExecutorService zkConnManagerCallbackExecutor = + private final ExecutorService zkConnectionListenerCallbackExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor( - new SolrNamedThreadFactory("zkConnectionManagerCallback")); + new SolrNamedThreadFactory("zkConnectionListenerCallback")); private volatile boolean isClosed = false; - private ZkClientConnectionStrategy zkClientConnectionStrategy; private int zkClientTimeout; - private ZkACLProvider zkACLProvider; + private ACLProvider aclProvider; private ZkCredentialsInjector zkCredentialsInjector; private String zkServerAddress; private SolrClassLoader solrClassLoader; @@ -113,10 +117,10 @@ public SolrZkClient(Builder builder) { builder.zkServerAddress, builder.zkClientTimeout, builder.zkClientConnectTimeout, - builder.connectionStrategy, + builder.zkCredentialsProvider, + builder.aclProvider, builder.onReconnect, builder.beforeReconnect, - builder.zkACLProvider, builder.higherLevelIsClosed, builder.minStateByteLenForCompression, builder.compressor, @@ -128,10 +132,10 @@ private SolrZkClient( String zkServerAddress, int zkClientTimeout, int clientConnectTimeout, - ZkClientConnectionStrategy strat, + ZkCredentialsProvider zkCredentialsProvider, + ACLProvider aclProvider, final OnReconnect onReconnect, BeforeReconnect beforeReconnect, - ZkACLProvider zkACLProvider, IsClosed higherLevelIsClosed, int minStateByteLenForCompression, Compressor compressor, @@ -143,87 +147,91 @@ private SolrZkClient( return; } this.zkServerAddress = zkServerAddress; - this.higherLevelIsClosed = higherLevelIsClosed; - if (strat == null) { - String connectionStrategy = System.getProperty("solr.zookeeper.connectionStrategy"); - strat = - ZkClientConnectionStrategy.forName(connectionStrategy, new DefaultConnectionStrategy()); + String chroot, zkHost; + int chrootIndex = zkServerAddress.indexOf('/'); + if (chrootIndex == -1) { + zkHost = zkServerAddress; + chroot = null; + } else if (chrootIndex == zkServerAddress.length() - 1) { + zkHost = zkServerAddress.substring(0, zkServerAddress.length() - 1); + chroot = null; + } else { + zkHost = zkServerAddress.substring(0, chrootIndex); + chroot = zkServerAddress.substring(chrootIndex + 1); } - this.zkClientConnectionStrategy = strat; + this.higherLevelIsClosed = higherLevelIsClosed; this.solrClassLoader = solrClassLoader; - if (!strat.hasZkCredentialsToAddAutomatically()) { + if (zkCredentialsProvider == null) { zkCredentialsInjector = useDefaultCredsAndACLs ? createZkCredentialsInjector() : new DefaultZkCredentialsInjector(); - ZkCredentialsProvider zkCredentialsToAddAutomatically = + zkCredentialsProvider = useDefaultCredsAndACLs ? createZkCredentialsToAddAutomatically() : new DefaultZkCredentialsProvider(); - strat.setZkCredentialsToAddAutomatically(zkCredentialsToAddAutomatically); + } + if (aclProvider == null) { + aclProvider = createACLProvider(); + } + if (chroot != null && aclProvider instanceof SecurityAwareZkACLProvider) { + this.aclProvider = ((SecurityAwareZkACLProvider) aclProvider).withChroot(chroot); + } else { + this.aclProvider = aclProvider; } this.zkClientTimeout = zkClientTimeout; - // we must retry at least as long as the session timeout - zkCmdExecutor = new ZkCmdExecutor(zkClientTimeout, SolrZkClient.this::isClosed); - connManager = - new ConnectionManager( - "ZooKeeperConnection Watcher:" + zkServerAddress, - this, - zkServerAddress, - strat, - onReconnect, - beforeReconnect, - SolrZkClient.this::isClosed); + curatorSafeServiceExecutor = + ExecutorUtil.newMDCAwareSingleThreadExecutor( + new SolrNamedThreadFactory("curator-safeService")); + + RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); + + client = + CuratorFrameworkFactory.builder() + .ensembleProvider(new FixedEnsembleProvider(zkHost)) + .namespace(chroot) + .sessionTimeoutMs(zkClientTimeout) + .connectionTimeoutMs(clientConnectTimeout) + .aclProvider(this.aclProvider) + .authorization(zkCredentialsProvider.getCredentials()) + .retryPolicy(retryPolicy) + .runSafeService(curatorSafeServiceExecutor) + .build(); + if (onReconnect != null) { + client + .getConnectionStateListenable() + .addListener(onReconnect, zkConnectionListenerCallbackExecutor); + } + if (beforeReconnect != null) { + client + .getConnectionStateListenable() + .addListener(beforeReconnect, zkConnectionListenerCallbackExecutor); + } + client.start(); try { - strat.connect( - zkServerAddress, - zkClientTimeout, - wrapWatcher(connManager), - zooKeeper -> { - ZooKeeper oldKeeper = keeper; - keeper = zooKeeper; - try { - closeKeeper(oldKeeper); - } finally { - if (isClosed) { - // we may have been closed - closeKeeper(SolrZkClient.this.keeper); - } - } - }); - } catch (Exception e) { - connManager.close(); - if (keeper != null) { - try { - keeper.close(); - } catch (InterruptedException e1) { - Thread.currentThread().interrupt(); - } + if (!client.blockUntilConnected(clientConnectTimeout, TimeUnit.MILLISECONDS)) { + throw new TimeoutException( + String.format( + Locale.ROOT, + "Timeout while waiting for Zookeeper Client to connect: %d ms", + clientConnectTimeout)); } - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); - } - - try { - connManager.waitForConnected(clientConnectTimeout); } catch (Exception e) { - connManager.close(); - try { - keeper.close(); - } catch (InterruptedException e1) { + if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - zkConnManagerCallbackExecutor.shutdown(); + client.close(); throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); } + assert ObjectReleaseTracker.track(this); - if (zkACLProvider == null) { - this.zkACLProvider = - useDefaultCredsAndACLs ? createZkACLProvider() : new DefaultZkACLProvider(); + if (aclProvider == null) { + this.aclProvider = useDefaultCredsAndACLs ? createACLProvider() : new DefaultZkACLProvider(); } else { - this.zkACLProvider = zkACLProvider; + this.aclProvider = aclProvider; } if (compressor == null) { @@ -234,12 +242,8 @@ private SolrZkClient( this.minStateByteLenForCompression = minStateByteLenForCompression; } - public ConnectionManager getConnectionManager() { - return connManager; - } - - public ZkClientConnectionStrategy getZkClientConnectionStrategy() { - return zkClientConnectionStrategy; + public CuratorFramework getCuratorFramework() { + return client; } public static final String ZK_CRED_PROVIDER_CLASS_NAME_VM_PARAM_NAME = "zkCredentialsProvider"; @@ -273,7 +277,7 @@ protected ZkCredentialsProvider createZkCredentialsToAddAutomatically() { public static final String ZK_ACL_PROVIDER_CLASS_NAME_VM_PARAM_NAME = "zkACLProvider"; - protected ZkACLProvider createZkACLProvider() { + protected ACLProvider createACLProvider() { String zkACLProviderClassName = System.getProperty(ZK_ACL_PROVIDER_CLASS_NAME_VM_PARAM_NAME); if (StrUtils.isNotNullOrEmpty(zkACLProviderClassName)) { try { @@ -336,20 +340,13 @@ protected ZkCredentialsInjector createZkCredentialsInjector() { /** Returns true if client is connected */ public boolean isConnected() { - return keeper != null && keeper.getState() == ZooKeeper.States.CONNECTED; + return client.getZookeeperClient().isConnected(); } public void delete(final String path, final int version, boolean retryOnConnLoss) throws InterruptedException, KeeperException { - if (retryOnConnLoss) { - zkCmdExecutor.retryOperation( - () -> { - keeper.delete(path, version); - return null; - }); - } else { - keeper.delete(path, version); - } + runWithCorrectThrows( + "deleting znode", () -> client.delete().withVersion(version).forPath(path)); metrics.deletes.increment(); } @@ -381,12 +378,10 @@ public Watcher wrapWatcher(final Watcher watcher) { */ public Stat exists(final String path, final Watcher watcher, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - Stat result = null; - if (retryOnConnLoss) { - result = zkCmdExecutor.retryOperation(() -> keeper.exists(path, wrapWatcher(watcher))); - } else { - result = keeper.exists(path, wrapWatcher(watcher)); - } + Stat result = + runWithCorrectThrows( + "checking exists", + () -> client.checkExists().usingWatcher(wrapWatcher(watcher)).forPath(path)); metrics.existsChecks.increment(); return result; } @@ -394,12 +389,8 @@ public Stat exists(final String path, final Watcher watcher, boolean retryOnConn /** Returns true if path exists */ public Boolean exists(final String path, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - Boolean result = null; - if (retryOnConnLoss) { - result = zkCmdExecutor.retryOperation(() -> keeper.exists(path, null) != null); - } else { - result = keeper.exists(path, null) != null; - } + Boolean result = + runWithCorrectThrows("checking exists", () -> client.checkExists().forPath(path) != null); metrics.existsChecks.increment(); return result; } @@ -407,12 +398,10 @@ public Boolean exists(final String path, boolean retryOnConnLoss) /** Returns children of the node at the path */ public List getChildren(final String path, final Watcher watcher, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - List result = null; - if (retryOnConnLoss) { - return zkCmdExecutor.retryOperation(() -> keeper.getChildren(path, wrapWatcher(watcher))); - } else { - result = keeper.getChildren(path, wrapWatcher(watcher)); - } + List result = + runWithCorrectThrows( + "getting children", + () -> client.getChildren().usingWatcher(wrapWatcher(watcher)).forPath(path)); metrics.childFetches.increment(); if (result != null) { @@ -425,13 +414,16 @@ public List getChildren(final String path, final Watcher watcher, boolea public List getChildren( final String path, final Watcher watcher, Stat stat, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - List result = null; - if (retryOnConnLoss) { - result = - zkCmdExecutor.retryOperation(() -> keeper.getChildren(path, wrapWatcher(watcher), stat)); - } else { - result = keeper.getChildren(path, wrapWatcher(watcher), stat); - } + List result = + runWithCorrectThrows( + "getting children", + () -> + client + .getChildren() + .storingStatIn(stat) + .usingWatcher(wrapWatcher(watcher)) + .forPath(path)); + metrics.childFetches.increment(); if (result != null) { metrics.cumulativeChildrenFetched.add(result.size()); @@ -443,12 +435,15 @@ public List getChildren( public byte[] getData( final String path, final Watcher watcher, final Stat stat, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - byte[] result = null; - if (retryOnConnLoss) { - result = zkCmdExecutor.retryOperation(() -> keeper.getData(path, wrapWatcher(watcher), stat)); - } else { - result = keeper.getData(path, wrapWatcher(watcher), stat); - } + byte[] result = + runWithCorrectThrows( + "getting data", + () -> + client + .getData() + .storingStatIn(stat) + .usingWatcher(wrapWatcher(watcher)) + .forPath(path)); if (compressor.isCompressedBytes(result)) { log.debug("Zookeeper data at path {} is compressed", path); try { @@ -477,17 +472,14 @@ public NodeData getNode(final String path, Watcher watcher, boolean retryOnConnL /** Returns node's state */ public Stat setData(final String path, byte data[], final int version, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - Stat result = null; if (SolrZkClient.shouldCompressData(data, path, minStateByteLenForCompression)) { // state.json should be compressed before being put to ZK data = compressor.compressBytes(data); } - if (retryOnConnLoss) { - byte[] finalData = data; - result = zkCmdExecutor.retryOperation(() -> keeper.setData(path, finalData, version)); - } else { - result = keeper.setData(path, data, version); - } + final byte[] finalData = data; + Stat result = + runWithCorrectThrows( + "setting data", () -> client.setData().withVersion(version).forPath(path, finalData)); metrics.writes.increment(); if (data != null) { metrics.bytesWritten.add(data.length); @@ -535,15 +527,9 @@ public void atomicUpdate(String path, BiFunction editor) public String create( final String path, final byte[] data, final CreateMode createMode, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - String result = null; - if (retryOnConnLoss) { - result = - zkCmdExecutor.retryOperation( - () -> keeper.create(path, data, zkACLProvider.getACLsToAdd(path), createMode)); - } else { - List acls = zkACLProvider.getACLsToAdd(path); - result = keeper.create(path, data, acls, createMode); - } + String result = + runWithCorrectThrows( + "creating znode", () -> client.create().withMode(createMode).forPath(path, data)); metrics.writes.increment(); if (data != null) { metrics.bytesWritten.add(data.length); @@ -563,13 +549,15 @@ public String create( boolean retryOnConnLoss, Stat stat) throws KeeperException, InterruptedException { - if (retryOnConnLoss) { - return zkCmdExecutor.retryOperation( - () -> keeper.create(path, data, zkACLProvider.getACLsToAdd(path), createMode, stat)); - } else { - List acls = zkACLProvider.getACLsToAdd(path); - return keeper.create(path, data, acls, createMode, stat); + String result = + runWithCorrectThrows( + "creating znode", + () -> client.create().storingStatIn(stat).withMode(createMode).forPath(path, data)); + metrics.writes.increment(); + if (data != null) { + metrics.bytesWritten.add(data.length); } + return result; } /** @@ -667,8 +655,7 @@ public void makePath( * *

skipPathParts will force the call to fail if the first skipPathParts do not exist already. * - *

Note: retryOnConnLoss is only respected for the final node - nodes before that are always - * retried on connection loss. + *

Note: retryOnConnLoss is no longer respected at all and is ignored * *

Note: if failOnExists == false then we will always overwrite the existing data with the * given data @@ -683,76 +670,77 @@ public void makePath( int skipPathParts) throws KeeperException, InterruptedException { log.debug("makePath: {}", path); - metrics.writes.increment(); - - boolean retry = true; - - if (path.startsWith("/")) { - path = path.substring(1, path.length()); + var createBuilder = client.create(); + if (!failOnExists) { + createBuilder.orSetData(); } + if (SolrZkClient.shouldCompressData(data, path, minStateByteLenForCompression)) { // state.json should be compressed before being put to ZK data = compressor.compressBytes(data); } + + metrics.writes.increment(); if (data != null) { metrics.bytesWritten.add(data.length); } - String[] paths = path.split("/"); - StringBuilder sbPath = new StringBuilder(); - for (int i = 0; i < paths.length; i++) { - String pathPiece = paths[i]; - sbPath.append("/").append(pathPiece); - if (i < skipPathParts) { - continue; + if (path.startsWith("/")) { + path = path.substring(1); + } + if (skipPathParts > 0) { + String[] paths = path.split("/"); + StringBuilder sbPath = new StringBuilder(); + for (int i = 0; i < paths.length; i++) { + if (i == skipPathParts) { + break; + } + String pathPiece = paths[i]; + sbPath.append("/").append(pathPiece); } - byte[] bytes = null; - final String currentPath = sbPath.toString(); - - CreateMode mode = CreateMode.PERSISTENT; - if (i == paths.length - 1) { - mode = createMode; - bytes = data; - if (!retryOnConnLoss) retry = false; + String skipPathPartsPath = sbPath.toString(); + final String finalSkipPathPartsPath; + if (!skipPathPartsPath.startsWith("/")) { + finalSkipPathPartsPath = "/" + skipPathPartsPath; + } else { + finalSkipPathPartsPath = skipPathPartsPath; } - try { - if (retry) { - final CreateMode finalMode = mode; - final byte[] finalBytes = bytes; - zkCmdExecutor.retryOperation( - () -> { - keeper.create( - currentPath, finalBytes, zkACLProvider.getACLsToAdd(currentPath), finalMode); - return null; - }); - } else { - keeper.create(currentPath, bytes, zkACLProvider.getACLsToAdd(currentPath), mode); - } - } catch (NoAuthException e) { - // in auth cases, we may not have permission for an earlier part of a path, which is fine - if (i == paths.length - 1 || !exists(currentPath, retryOnConnLoss)) { - - throw e; - } - } catch (NodeExistsException e) { - if (log.isDebugEnabled()) { - log.debug("Node exists: {}", e.getPath()); - } - - if (!failOnExists && i == paths.length - 1) { - // TODO: version ? for now, don't worry about race - setData(currentPath, data, -1, retryOnConnLoss); - // set new watch - exists(currentPath, watcher, retryOnConnLoss); - return; - } - - // ignore unless it's the last node in the path - if (i == paths.length - 1) { - throw e; - } + Stat exists = + runWithCorrectThrows( + "checking skipPathParts exists", + () -> client.checkExists().forPath(finalSkipPathPartsPath)); + if (exists == null) { + throw new KeeperException.NoNodeException(finalSkipPathPartsPath + " didn't already exist"); } } + + final String finalPath; + if (!path.startsWith("/")) { + finalPath = "/" + path; + } else { + finalPath = path; + } + + final byte[] finalData = data; + runWithCorrectThrows( + "making path", + () -> { + // This try/catch can be removed when CURATOR-719 is incorporated + try { + createBuilder + .creatingParentsIfNeeded() + .withMode(createMode) + .forPath(finalPath, finalData); + } catch (KeeperException.NodeExistsException e) { + // There can be a race exception when this is called in parallel + if (!failOnExists) { + setData(finalPath, finalData, true); + } else { + throw e; + } + } + return client.checkExists().usingWatcher(wrapWatcher(watcher)).forPath(finalPath); + }); } public void makePath( @@ -761,6 +749,72 @@ public void makePath( makePath(zkPath, null, createMode, watcher, retryOnConnLoss); } + /** + * Create a node if it does not exist + * + * @param path the path at which to create the znode + */ + public void ensureExists(final String path) throws KeeperException, InterruptedException { + ensureExists(path, null); + } + + /** + * Create a node if it does not exist + * + * @param path the path at which to create the znode + * @param data the optional data to set on the znode + */ + public void ensureExists(final String path, final byte[] data) + throws KeeperException, InterruptedException { + ensureExists(path, data, CreateMode.PERSISTENT); + } + + /** + * Create a node if it does not exist + * + * @param path the path at which to create the znode + * @param data the optional data to set on the znode + * @param createMode the mode with which to create the znode + */ + public void ensureExists(final String path, final byte[] data, CreateMode createMode) + throws KeeperException, InterruptedException { + ensureExists(path, data, createMode, 0); + } + + /** + * Create a node if it does not exist + * + * @param path the path at which to create the znode + * @param data the optional data to set on the znode + * @param createMode the mode with which to create the znode + * @param skipPathParts how many path elements to skip + */ + public void ensureExists( + final String path, final byte[] data, CreateMode createMode, int skipPathParts) + throws KeeperException, InterruptedException { + if (exists(path, true)) { + return; + } + try { + if (skipPathParts > 0) { + int endingIndex = 0; + for (int i = 0; i < skipPathParts && endingIndex >= 0; i++) { + endingIndex = path.indexOf('/', endingIndex + 1); + } + if (endingIndex == -1 || endingIndex == path.length() - 1) { + throw new KeeperException.NoNodeException(path); + } + String startingPath = path.substring(0, endingIndex); + if (!exists(startingPath, true)) { + throw new KeeperException.NoNodeException(startingPath); + } + } + makePath(path, data, createMode, null, true, true, skipPathParts); + } catch (KeeperException.NodeExistsException ignored) { + // it's okay if another beats us creating the node + } + } + /** Write data to ZooKeeper. */ public Stat setData(String path, byte[] data, boolean retryOnConnLoss) throws KeeperException, InterruptedException { @@ -781,14 +835,44 @@ public Stat setData(String path, Path source, boolean retryOnConnLoss) return setData(path, Files.readAllBytes(source), retryOnConnLoss); } - public List multi(final Iterable ops, boolean retryOnConnLoss) - throws InterruptedException, KeeperException { - List result = null; - if (retryOnConnLoss) { - result = zkCmdExecutor.retryOperation(() -> keeper.multi(ops)); - } else { - result = keeper.multi(ops); + /** + * A function that takes a transaction builder and returns a curator operation. This is used for + * users to provide a multi-transaction operation to the client, without needing to use the + * underlying Curator client to access the transaction builder. + */ + @FunctionalInterface + public interface CuratorOpBuilder { + CuratorOp build(TransactionOp startingOp) throws Exception; + + default CuratorOp buildWithoutThrows(TransactionOp startingOp) { + try { + return build(startingOp); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new RuntimeException(e); + } } + } + + public List multi(CuratorOpBuilder... ops) + throws InterruptedException, KeeperException { + return multi(Arrays.asList(ops)); + } + + public List multi(final List ops) + throws InterruptedException, KeeperException { + List result = + runWithCorrectThrows( + "executing multi-transaction", + () -> + client + .transaction() + .forOperations( + ops.stream() + .map(op -> op.buildWithoutThrows(client.transactionOp())) + .collect(Collectors.toList()))); + metrics.multiOps.increment(); if (result != null) { metrics.cumulativeMultiOps.add(result.size()); @@ -802,11 +886,9 @@ public void printLayout(String path, int indent, StringBuilder string) byte[] data = getData(path, null, null, true); List children = getChildren(path, null, true); StringBuilder dent = new StringBuilder(); - for (int i = 0; i < indent; i++) { - dent.append(" "); - } + dent.append(" ".repeat(Math.max(0, indent))); string.append(dent).append(path).append(" (").append(children.size()).append(")").append(NEWL); - if (data != null) { + if (data != null && data.length > 0) { String dataString = new String(data, StandardCharsets.UTF_8); if (!path.endsWith(".txt") && !path.endsWith(".xml")) { string @@ -845,8 +927,8 @@ public void close() { try { closeCallbackExecutor(); } finally { - connManager.close(); - closeKeeper(keeper); + IOUtils.closeQuietly(client); + ExecutorUtil.shutdownNowAndAwaitTermination(curatorSafeServiceExecutor); } assert ObjectReleaseTracker.release(this); } @@ -855,32 +937,24 @@ public boolean isClosed() { return isClosed || (higherLevelIsClosed != null && higherLevelIsClosed.isClosed()); } - /** Allows package private classes to update volatile ZooKeeper. */ - void updateKeeper(ZooKeeper keeper) throws InterruptedException { - ZooKeeper oldKeeper = this.keeper; - this.keeper = keeper; - if (oldKeeper != null) { - oldKeeper.close(); + public long getZkSessionId() { + if (isConnected()) { + try { + return client.getZookeeperClient().getZooKeeper().getSessionId(); + } catch (Exception ignored) { + } } - // we might have been closed already - if (isClosed) this.keeper.close(); + return -1; } - public ZooKeeper getZooKeeper() { - return keeper; - } - - private void closeKeeper(ZooKeeper keeper) { - if (keeper != null) { + public int getZkSessionTimeout() { + if (isConnected()) { try { - keeper.close(); - } catch (InterruptedException e) { - // Restore the interrupted status - Thread.currentThread().interrupt(); - log.error("", e); - throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e); + return client.getZookeeperClient().getLastNegotiatedSessionTimeoutMs(); + } catch (Exception ignored) { } } + return 0; } private void closeCallbackExecutor() { @@ -891,7 +965,7 @@ private void closeCallbackExecutor() { } try { - ExecutorUtil.shutdownAndAwaitTermination(zkConnManagerCallbackExecutor); + ExecutorUtil.shutdownAndAwaitTermination(zkConnectionListenerCallbackExecutor); } catch (Exception e) { log.error("Error shutting down zkConnManagerCallbackExecutor", e); } @@ -924,6 +998,20 @@ public String getZkServerAddress() { return zkServerAddress; } + /** + * @return the ChRoot that this client is using for the ZK cluster + */ + public String getChroot() { + return client.getNamespace(); + } + + /** + * @return the given path's absolute path in the ZK cluster + */ + public String getAbsolutePath(String path) { + return ZKPaths.fixForNamespace(getChroot(), path); + } + /** * Gets the raw config node /zookeeper/config as returned by server. Response may look like * @@ -938,25 +1026,17 @@ public String getZkServerAddress() { * string */ public String getConfig() { - try { - Stat stat = new Stat(); - keeper.sync(ZooDefs.CONFIG_NODE, null, null); - byte[] data = keeper.getConfig(false, stat); - if (data == null || data.length == 0) { - return ""; - } - return new String(data, StandardCharsets.UTF_8); - } catch (NoNodeException nne) { + QuorumVerifier currentConfig = client.getCurrentConfig(); + if (currentConfig == null) { log.debug("Zookeeper does not have the /zookeeper/config znode, assuming old ZK version"); return ""; - } catch (KeeperException | InterruptedException ex) { - throw new SolrException( - SolrException.ErrorCode.SERVER_ERROR, "Failed to get config from zookeeper", ex); + } else { + return currentConfig.toString(); } } - public ZkACLProvider getZkACLProvider() { - return zkACLProvider; + public ACLProvider getZkACLProvider() { + return aclProvider; } /** @@ -964,11 +1044,8 @@ public ZkACLProvider getZkACLProvider() { */ public List getACL(String path, Stat stat, boolean retryOnConnLoss) throws KeeperException, InterruptedException { - if (retryOnConnLoss) { - return zkCmdExecutor.retryOperation(() -> keeper.getACL(path, stat)); - } else { - return keeper.getACL(path, stat); - } + return runWithCorrectThrows( + "getting acls", () -> client.getACL().storingStatIn(stat).forPath(path)); } /** @@ -976,16 +1053,11 @@ public List getACL(String path, Stat stat, boolean retryOnConnLoss) * * @param path path to set ACL on e.g. /solr/conf/solrconfig.xml * @param acls a list of {@link ACL}s to be applied - * @param retryOnConnLoss true if the command should be retried on connection loss * @return the stat of the node */ - public Stat setACL(String path, List acls, boolean retryOnConnLoss) - throws InterruptedException, KeeperException { - if (retryOnConnLoss) { - return zkCmdExecutor.retryOperation(() -> keeper.setACL(path, acls, -1)); - } else { - return keeper.setACL(path, acls, -1); - } + public Stat setACL(String path, List acls) throws InterruptedException, KeeperException { + return runWithCorrectThrows( + "setting acls", () -> client.setACL().withVersion(-1).withACL(acls).forPath(path)); } /** @@ -1000,14 +1072,33 @@ public void updateACLs(final String root) throws KeeperException, InterruptedExc ZkMaintenanceUtils.VISIT_ORDER.VISIT_POST, path -> { try { - setACL(path, getZkACLProvider().getACLsToAdd(path), true); - log.debug("Updated ACL on {}", path); + runWithCorrectThrows( + "updating acls", () -> client.setACL().withACL(null).forPath(path)); } catch (NoNodeException ignored) { // If a node was deleted, don't bother trying to set ACLs on it. } }); } + @FunctionalInterface + protected interface SupplierWithException { + T get() throws Exception; + } + + protected T runWithCorrectThrows(String action, SupplierWithException func) + throws KeeperException, InterruptedException { + try { + return func.get(); + } catch (KeeperException | RuntimeException e) { + throw e; + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw e; + } catch (Exception e) { + throw new RuntimeException("Exception occurred while " + action, e); + } + } + // Some pass-throughs to allow less code disruption to other classes that use SolrZkClient. public void clean(String path) throws InterruptedException, KeeperException { ZkMaintenanceUtils.clean(this, path); @@ -1055,6 +1146,11 @@ public void downloadFromZK(String zkPath, Path dir) throws IOException { ZkMaintenanceUtils.downloadFromZK(this, zkPath, dir); } + @FunctionalInterface + public interface IsClosed { + boolean isClosed(); + } + /** * Watcher wrapper that ensures that heavy implementations of process do not interfere with our * ability to react to other watches, but also ensures that two wrappers containing equal watches @@ -1074,15 +1170,11 @@ private final class ProcessWatchWithExecutor implements Watcher { // see below f public void process(final WatchedEvent event) { log.debug("Submitting job to respond to event {}", event); try { - if (watcher instanceof ConnectionManager) { - zkConnManagerCallbackExecutor.execute(() -> watcher.process(event)); - } else { - zkCallbackExecutor.execute( - () -> { - metrics.watchesFired.increment(); - watcher.process(event); - }); - } + zkCallbackExecutor.execute( + () -> { + metrics.watchesFired.increment(); + watcher.process(event); + }); } catch (RejectedExecutionException e) { // If not a graceful shutdown if (!isClosed()) { @@ -1166,8 +1258,8 @@ public static class Builder { public int zkClientConnectTimeout = SolrZkClientTimeout.DEFAULT_ZK_CONNECT_TIMEOUT; public OnReconnect onReconnect; public BeforeReconnect beforeReconnect; - public ZkClientConnectionStrategy connectionStrategy; - public ZkACLProvider zkACLProvider; + public ZkCredentialsProvider zkCredentialsProvider; + public ACLProvider aclProvider; public IsClosed higherLevelIsClosed; public SolrClassLoader solrClassLoader; public boolean useDefaultCredsAndACLs = true; @@ -1221,18 +1313,18 @@ public Builder withReconnectListener(OnReconnect onReconnect) { return this; } - public Builder withConnStrategy(ZkClientConnectionStrategy strat) { - this.connectionStrategy = strat; + public Builder withBeforeConnect(BeforeReconnect beforeReconnect) { + this.beforeReconnect = beforeReconnect; return this; } - public Builder withBeforeConnect(BeforeReconnect beforeReconnect) { - this.beforeReconnect = beforeReconnect; + public Builder withZkCredentialsProvider(ZkCredentialsProvider zkCredentialsProvider) { + this.zkCredentialsProvider = zkCredentialsProvider; return this; } - public Builder withAclProvider(ZkACLProvider zkACLProvider) { - this.zkACLProvider = zkACLProvider; + public Builder withAclProvider(ACLProvider aclProvider) { + this.aclProvider = aclProvider; return this; } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/VMParamsSingleSetCredentialsDigestZkCredentialsProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/VMParamsSingleSetCredentialsDigestZkCredentialsProvider.java index 0674a088616..05c3d88d199 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/VMParamsSingleSetCredentialsDigestZkCredentialsProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/VMParamsSingleSetCredentialsDigestZkCredentialsProvider.java @@ -16,7 +16,8 @@ */ package org.apache.solr.common.cloud; -import java.util.Collection; +import java.util.List; +import org.apache.curator.framework.AuthInfo; /** * Deprecated in favor of a combination of {@link DigestZkCredentialsProvider} and {@link @@ -64,7 +65,7 @@ public void setZkCredentialsInjector(ZkCredentialsInjector zkCredentialsInjector } @Override - protected Collection createCredentials() { + protected List createCredentials() { return digestZkCredentialsProvider.createCredentials(); } } diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkACLProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkACLProvider.java index 9c1fab51766..4364d0ba236 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkACLProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkACLProvider.java @@ -17,12 +17,18 @@ package org.apache.solr.common.cloud; import java.util.List; +import org.apache.curator.framework.api.ACLProvider; import org.apache.zookeeper.data.ACL; -public interface ZkACLProvider { +public interface ZkACLProvider extends ACLProvider { List getACLsToAdd(String zNodePath); + @Override + default List getAclForPath(String zNodePath) { + return getACLsToAdd(zNodePath); + } + /** * @param zkCredentialsInjector The ZkCredentialsInjector that injects ZK credentials */ diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java deleted file mode 100644 index 8102606a59b..00000000000 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java +++ /dev/null @@ -1,155 +0,0 @@ -/* - * 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 java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeoutException; -import org.apache.solr.common.cloud.ZkCredentialsProvider.ZkCredentials; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** */ -public abstract class ZkClientConnectionStrategy { - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - private volatile ZkCredentialsProvider zkCredentialsToAddAutomatically; - private volatile boolean zkCredentialsToAddAutomaticallyUsed; - - private List disconnectedListeners = new ArrayList<>(); - private List connectedListeners = new ArrayList<>(); - - public abstract void connect( - String zkServerAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) - throws IOException, InterruptedException, TimeoutException; - - public abstract void reconnect( - String serverAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) - throws IOException, InterruptedException, TimeoutException; - - public ZkClientConnectionStrategy() { - zkCredentialsToAddAutomaticallyUsed = false; - } - - public synchronized void disconnected() { - for (DisconnectedListener listener : disconnectedListeners) { - try { - listener.disconnected(); - } catch (Exception e) { - log.error("Exception on disconnected for listener: {}", listener, e); - } - } - } - - public synchronized void connected() { - for (ConnectedListener listener : connectedListeners) { - try { - listener.connected(); - } catch (Exception e) { - log.error("Exception on connected for listener: {}", listener, e); - } - } - } - - public interface DisconnectedListener { - void disconnected(); - } - - public interface ConnectedListener { - void connected(); - } - - public synchronized void addDisconnectedListener(DisconnectedListener listener) { - disconnectedListeners.add(listener); - } - - public synchronized void removeDisconnectedListener(DisconnectedListener listener) { - disconnectedListeners.remove(listener); - } - - public synchronized void addConnectedListener(ConnectedListener listener) { - connectedListeners.add(listener); - } - - public interface ZkUpdate { - void update(ZooKeeper zooKeeper) throws InterruptedException, TimeoutException, IOException; - } - - public void setZkCredentialsToAddAutomatically( - ZkCredentialsProvider zkCredentialsToAddAutomatically) { - if (zkCredentialsToAddAutomaticallyUsed || (zkCredentialsToAddAutomatically == null)) - throw new RuntimeException( - "Cannot change zkCredentialsToAddAutomatically after it has been (connect or reconnect was called) used or to null"); - this.zkCredentialsToAddAutomatically = zkCredentialsToAddAutomatically; - } - - public boolean hasZkCredentialsToAddAutomatically() { - return zkCredentialsToAddAutomatically != null; - } - - public ZkCredentialsProvider getZkCredentialsToAddAutomatically() { - return zkCredentialsToAddAutomatically; - } - - protected ZooKeeper createZooKeeper( - final String serverAddress, final int zkClientTimeout, final Watcher watcher) - throws IOException { - ZooKeeper result = newZooKeeperInstance(serverAddress, zkClientTimeout, watcher); - - zkCredentialsToAddAutomaticallyUsed = true; - for (ZkCredentials zkCredentials : zkCredentialsToAddAutomatically.getCredentials()) { - result.addAuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth()); - } - - return result; - } - - // Override for testing - protected ZooKeeper newZooKeeperInstance( - final String serverAddress, final int zkClientTimeout, final Watcher watcher) - throws IOException { - return new ZooKeeper(serverAddress, zkClientTimeout, watcher); - } - - /** - * Instantiate a new connection strategy for the given class name - * - * @param name the name of the strategy to use - * @return the strategy instance, or null if it could not be loaded - */ - public static ZkClientConnectionStrategy forName(String name, ZkClientConnectionStrategy def) { - log.debug("Attempting to load zk connection strategy '{}'", name); - if (name == null) { - return def; - } - - try { - // TODO should this use SolrResourceLoader? - return Class.forName(name) - .asSubclass(ZkClientConnectionStrategy.class) - .getConstructor() - .newInstance(); - } catch (Exception e) { - log.warn("Exception when loading '{}' ZK connection strategy.", name, e); - return def; - } - } -} diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java deleted file mode 100644 index 01924b3c044..00000000000 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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 java.lang.invoke.MethodHandles; -import org.apache.solr.common.AlreadyClosedException; -import org.apache.solr.common.cloud.ConnectionManager.IsClosed; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ZkCmdExecutor { - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - private long retryDelay = 1500L; // 1 second would match timeout, so 500 ms over for padding - private int retryCount; - private double timeouts; - private IsClosed isClosed; - - public ZkCmdExecutor(int timeoutms) { - this(timeoutms, null); - } - - /** - * TODO: At this point, this should probably take a SolrZkClient in its constructor. - * - * @param timeoutms the client timeout for the ZooKeeper clients that will be used with this - * class. - */ - public ZkCmdExecutor(int timeoutms, IsClosed isClosed) { - timeouts = timeoutms / 1000.0; - this.retryCount = Math.round(0.5f * ((float) Math.sqrt(8.0f * timeouts + 1.0f) - 1.0f)) + 1; - this.isClosed = isClosed; - } - - public long getRetryDelay() { - return retryDelay; - } - - public void setRetryDelay(long retryDelay) { - this.retryDelay = retryDelay; - } - - /** Perform the given operation, retrying if the connection fails */ - public T retryOperation(ZkOperation operation) - throws KeeperException, InterruptedException { - KeeperException exception = null; - for (int i = 0; i < retryCount; i++) { - try { - if (log.isTraceEnabled()) { - log.trace("Begin zookeeper operation {}, attempt={}", operation, i); - } - if (i > 0 && isClosed()) { - throw new AlreadyClosedException(); - } - return operation.execute(); - } catch (KeeperException.ConnectionLossException e) { - if (exception == null) { - exception = e; - } - if (Thread.currentThread().isInterrupted()) { - Thread.currentThread().interrupt(); - throw new InterruptedException(); - } - if (i != retryCount - 1) { - retryDelay(i); - } - } finally { - if (log.isTraceEnabled()) { - log.trace("End zookeeper operation {}", operation); - } - } - } - throw exception; - } - - private boolean isClosed() { - return isClosed != null && isClosed.isClosed(); - } - - /** - * Performs a retry delay if this is not the first attempt - * - * @param attemptCount the number of the attempts performed so far - */ - protected void retryDelay(int attemptCount) throws InterruptedException { - Thread.sleep((attemptCount + 1) * retryDelay); - } -} diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java index 8614ce6880c..24091258593 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java @@ -16,30 +16,11 @@ */ package org.apache.solr.common.cloud; -import java.util.Collection; +import java.util.List; +import org.apache.curator.framework.AuthInfo; public interface ZkCredentialsProvider { - - class ZkCredentials { - String scheme; - byte[] auth; - - public ZkCredentials(String scheme, byte[] auth) { - super(); - this.scheme = scheme; - this.auth = auth; - } - - public String getScheme() { - return scheme; - } - - public byte[] getAuth() { - return auth; - } - } - - Collection getCredentials(); + List getCredentials(); void setZkCredentialsInjector(ZkCredentialsInjector zkCredentialsInjector); } 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 0604f00768f..df638085a2e 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 @@ -573,13 +573,18 @@ public synchronized void createClusterStateWatchersAndUpdate() } collectionPropertiesZkStateReader.refreshCollectionProperties(); } catch (KeeperException.NoNodeException nne) { + String noNodePath = nne.getPath(); + if (noNodePath.length() > zkClient.getCuratorFramework().getNamespace().length()) { + noNodePath = + noNodePath.substring(zkClient.getCuratorFramework().getNamespace().length() + 1); + } throw new SolrException( ErrorCode.SERVICE_UNAVAILABLE, "Cannot connect to cluster at " + zkClient.getZkServerAddress() + ": cluster not found/not ready." + " Expected node '" - + nne.getPath() + + noNodePath + "' does not exist."); } } @@ -2032,7 +2037,8 @@ public boolean update() throws KeeperException, InterruptedException { log.debug("Checking ZK for most up to date Aliases {}", ALIASES); } // Call sync() first to ensure the subsequent read (getData) is up-to-date. - zkClient.getZooKeeper().sync(ALIASES, null, null); + zkClient.runWithCorrectThrows( + "syncing aliases", () -> zkClient.getCuratorFramework().sync().forPath(ALIASES)); return setIfNewer(zkClient.getNode(ALIASES, null, true)); } diff --git a/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java b/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java index 6e59fe3972a..9a3429520c0 100644 --- a/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java +++ b/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java @@ -21,13 +21,15 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.security.NoSuchAlgorithmException; -import java.util.Collection; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.curator.framework.AuthInfo; +import org.apache.curator.framework.api.ACLProvider; import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.cloud.AbstractZkTestCase; @@ -102,18 +104,17 @@ public void setUp() throws Exception { .withUrl(zkServer.getZkAddress()) .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)) { @Override - protected ZkACLProvider createZkACLProvider() { - return new DefaultZkACLProvider() { - @Override - protected List createGlobalACLsToAdd() { - try { - Id id = new Id(SCHEME, DigestAuthenticationProvider.generateDigest(AUTH)); - return Collections.singletonList(new ACL(ZooDefs.Perms.ALL, id)); - } catch (NoSuchAlgorithmException e) { - throw new RuntimeException(e); - } - } - }; + protected ACLProvider createACLProvider() { + try { + // Must be Arrays.asList(), Zookeeper does not allow for immutable list types for ACLs + return new DefaultZkACLProvider( + Arrays.asList( + new ACL( + ZooDefs.Perms.ALL, + new Id(SCHEME, DigestAuthenticationProvider.generateDigest(AUTH))))); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } } }; @@ -124,13 +125,9 @@ protected List createGlobalACLsToAdd() { .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)) { @Override protected ZkCredentialsProvider createZkCredentialsToAddAutomatically() { - return new DefaultZkCredentialsProvider() { - @Override - protected Collection createCredentials() { - return Collections.singleton( - new ZkCredentials(SCHEME, AUTH.getBytes(StandardCharsets.UTF_8))); - } - }; + return new DefaultZkCredentialsProvider( + Collections.singletonList( + new AuthInfo(SCHEME, AUTH.getBytes(StandardCharsets.UTF_8)))); } }; } @@ -273,7 +270,7 @@ private static boolean canRead(SolrZkClient zkClient, String path) public void getConfig() { // As the embedded ZK is hardcoded to standalone, there is no way to test actual config data // here - assertEquals("", defaultClient.getConfig()); + assertEquals("version=0", defaultClient.getConfig()); } @Test diff --git a/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java b/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java index 8a2ff9f84cf..8dc77b154e2 100644 --- a/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java +++ b/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java @@ -22,11 +22,12 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.security.NoSuchAlgorithmException; -import java.util.ArrayList; -import java.util.Collection; +import java.util.Arrays; import java.util.List; import java.util.Properties; import java.util.concurrent.TimeUnit; +import org.apache.curator.framework.AuthInfo; +import org.apache.curator.framework.api.ACLProvider; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.cloud.AbstractZkTestCase; import org.apache.solr.cloud.ZkConfigSetService; @@ -143,7 +144,7 @@ public void testUploadConfig() throws IOException { } @Test - public void testUploadWithACL() throws IOException { + public void testUploadWithACL() throws IOException, NoSuchAlgorithmException { zkServer.ensurePathExists("/acl"); @@ -152,59 +153,36 @@ public void testUploadWithACL() throws IOException { final String writeableUsername = "writeable"; final String writeablePassword = "writeable"; - ZkACLProvider aclProvider = - new DefaultZkACLProvider() { - @Override - protected List createGlobalACLsToAdd() { - try { - List result = new ArrayList<>(); - result.add( - new ACL( - ZooDefs.Perms.ALL, - new Id( - "digest", - DigestAuthenticationProvider.generateDigest( - writeableUsername + ":" + writeablePassword)))); - result.add( - new ACL( - ZooDefs.Perms.READ, - new Id( - "digest", - DigestAuthenticationProvider.generateDigest( - readOnlyUsername + ":" + readOnlyPassword)))); - return result; - } catch (NoSuchAlgorithmException e) { - throw new RuntimeException(e); - } - } - }; - - ZkCredentialsProvider readonly = - new DefaultZkCredentialsProvider() { - @Override - protected Collection createCredentials() { - List credentials = new ArrayList<>(); - credentials.add( - new ZkCredentials( + // Must be Arrays.asList(), Zookeeper does not allow for immutable list types for ACLs + List acls = + Arrays.asList( + new ACL( + ZooDefs.Perms.ALL, + new Id( "digest", - (readOnlyUsername + ":" + readOnlyPassword).getBytes(StandardCharsets.UTF_8))); - return credentials; - } - }; - - ZkCredentialsProvider writeable = - new DefaultZkCredentialsProvider() { - @Override - protected Collection createCredentials() { - List credentials = new ArrayList<>(); - credentials.add( - new ZkCredentials( + DigestAuthenticationProvider.generateDigest( + writeableUsername + ":" + writeablePassword))), + new ACL( + ZooDefs.Perms.READ, + new Id( "digest", - (writeableUsername + ":" + writeablePassword) - .getBytes(StandardCharsets.UTF_8))); - return credentials; - } - }; + DigestAuthenticationProvider.generateDigest( + readOnlyUsername + ":" + readOnlyPassword)))); + ACLProvider aclProvider = new DefaultZkACLProvider(acls); + + List credentials = + List.of( + new AuthInfo( + "digest", + (readOnlyUsername + ":" + readOnlyPassword).getBytes(StandardCharsets.UTF_8))); + ZkCredentialsProvider readonly = new DefaultZkCredentialsProvider(credentials); + + List writeableCredentials = + List.of( + new AuthInfo( + "digest", + (writeableUsername + ":" + writeablePassword).getBytes(StandardCharsets.UTF_8))); + ZkCredentialsProvider writeable = new DefaultZkCredentialsProvider(writeableCredentials); Path configPath = createTempDir("acl-config"); Files.createFile(configPath.resolve("file1")); @@ -265,7 +243,7 @@ public void testBootstrapConf() throws IOException, KeeperException, Interrupted static SolrZkClient buildZkClient( String zkAddress, - final ZkACLProvider aclProvider, + final ACLProvider aclProvider, final ZkCredentialsProvider credentialsProvider) { return new SolrZkClient( new SolrZkClient.Builder().withUrl(zkAddress).withTimeout(10000, TimeUnit.MILLISECONDS)) { @@ -275,7 +253,7 @@ protected ZkCredentialsProvider createZkCredentialsToAddAutomatically() { } @Override - protected ZkACLProvider createZkACLProvider() { + protected ACLProvider createACLProvider() { return aclProvider; } }; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrZkClientTimeout.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrZkClientTimeout.java index 0a0cfd7fdad..a46f269060b 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrZkClientTimeout.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrZkClientTimeout.java @@ -18,6 +18,8 @@ public class SolrZkClientTimeout { public static final int DEFAULT_ZK_CLIENT_TIMEOUT = Integer.getInteger("zkClientTimeout", 30000); + // Make this overridable for tests. A lot take 15 seconds to timeout because of this. + // or CURATOR-720 public static final int DEFAULT_ZK_CONNECT_TIMEOUT = Integer.getInteger("zkConnectTimeout", 15000); diff --git a/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java index 334851ca985..21a5307eb67 100644 --- a/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java +++ b/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java @@ -51,6 +51,7 @@ + "org.apache.solr.common.cloud=INFO;" + "org.apache.solr.cloud.api.collections=INFO;" + "org.apache.solr.cloud.overseer=INFO") +@SolrCloudTestCase.NoPrs public class PerReplicaStatesIntegrationTest extends SolrCloudTestCase { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); diff --git a/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientCompressedDataTest.java b/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientCompressedDataTest.java index 82682e5e122..dbdbbc39449 100644 --- a/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientCompressedDataTest.java +++ b/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientCompressedDataTest.java @@ -75,11 +75,8 @@ public void getData() throws Exception { + "\"leader\":\"true\"}}}}}}"; byte[] arr = state.getBytes(StandardCharsets.UTF_8); byte[] compressedData = zLibStateCompression.compressBytes(arr); - ZkACLProvider aclProvider = new DefaultZkACLProvider(); String path = ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json"; - zkClient - .getZooKeeper() - .create(path, compressedData, aclProvider.getACLsToAdd(path), CreateMode.PERSISTENT); + zkClient.create(path, compressedData, CreateMode.PERSISTENT, true); byte[] data = zkClient.getData(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", null, null, true); diff --git a/solr/test-framework/build.gradle b/solr/test-framework/build.gradle index cb0ddabf7fa..2207ccbe515 100644 --- a/solr/test-framework/build.gradle +++ b/solr/test-framework/build.gradle @@ -43,6 +43,18 @@ dependencies { var zkExcludes = { exclude group: "org.apache.yetus", module: "audience-annotations" } + implementation('org.apache.curator:curator-client', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) + implementation('org.apache.curator:curator-framework', { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + }) + // Necessary because ChaosMonkey uses KillSession from curator-test + implementation('org.apache.curator:curator-test') { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + exclude group: 'com.google.guava', module: 'guava' + exclude group: 'io.dropwizard.metrics', module: 'metrics-core' + } implementation('org.apache.zookeeper:zookeeper', zkExcludes) implementation('org.apache.zookeeper:zookeeper-jute', zkExcludes) diff --git a/solr/test-framework/src/java/org/apache/solr/SolrIgnoredThreadsFilter.java b/solr/test-framework/src/java/org/apache/solr/SolrIgnoredThreadsFilter.java index a3903e8ceaa..dc0e22dc6a3 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrIgnoredThreadsFilter.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrIgnoredThreadsFilter.java @@ -82,6 +82,11 @@ public boolean reject(Thread t) { return true; } + // CURATOR-720 + if (threadName.equals("OverseerExitThread")) { + return true; + } + return threadName.startsWith("closeThreadPool"); } } diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java index 035fbcdef7f..27e9d7a9c33 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java @@ -94,7 +94,6 @@ import org.apache.solr.client.solrj.util.ClientUtils; import org.apache.solr.cloud.IpTables; import org.apache.solr.cloud.MiniSolrCloudCluster; -import org.apache.solr.cloud.TestConnectionStrategy; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrDocumentList; import org.apache.solr.common.SolrException; @@ -286,7 +285,6 @@ public static void setupTestCases() { System.setProperty("zookeeper.forceSync", "no"); System.setProperty("jetty.testMode", "true"); - System.setProperty("solr.zookeeper.connectionStrategy", TestConnectionStrategy.class.getName()); System.setProperty("enable.update.log", Boolean.toString(usually())); System.setProperty("tests.shardhandler.randomSeed", Long.toString(random().nextLong())); System.setProperty("solr.clustering.enabled", "false"); diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDigestZkACLAndCredentialsProvidersTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDigestZkACLAndCredentialsProvidersTestBase.java index dbbf308bad1..31bfdb837c6 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDigestZkACLAndCredentialsProvidersTestBase.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDigestZkACLAndCredentialsProvidersTestBase.java @@ -134,6 +134,8 @@ public void setUp() throws Exception { // no (or completely open) ACLs added. Therefore, hack your way into being authorized for // creating anyway zkClient + .getCuratorFramework() + .getZookeeperClient() .getZooKeeper() .addAuthInfo( "digest", (ALL_USERNAME + ":" + ALL_PASSWORD).getBytes(StandardCharsets.UTF_8)); @@ -305,6 +307,8 @@ public void testRepairACL() throws Exception { // no (or completely open) ACLs added. Therefore, hack your way into being authorized for // creating anyway zkClient + .getCuratorFramework() + .getZookeeperClient() .getZooKeeper() .addAuthInfo( "digest", @@ -336,7 +340,7 @@ public void testRepairACL() throws Exception { NoAuthException e = assertThrows( NoAuthException.class, () -> zkClient.getData("/security.json", null, null, false)); - assertEquals("/security.json", e.getPath()); + assertEquals("/solr/security.json", e.getPath()); } } diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java index 01d07877a24..805866dc855 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java @@ -16,7 +16,6 @@ */ package org.apache.solr.cloud; -import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.List; @@ -29,6 +28,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; +import org.apache.curator.test.KillSession; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.solr.cloud.AbstractFullDistribZkTestBase.CloudJettyRunner; import org.apache.solr.common.cloud.DocCollection; @@ -48,8 +48,6 @@ import org.apache.solr.util.TestInjection; import org.apache.solr.util.TimeOut; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.TestableZooKeeper; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -151,8 +149,9 @@ public void expireSession(final JettySolrRunner jetty) { CoreContainer cores = jetty.getCoreContainer(); if (cores != null) { monkeyLog("expire session for " + jetty.getLocalPort() + " !"); + long sessionId = cores.getZkController().getZkClient().getZkSessionId(); + zkServer.expire(sessionId); causeConnectionLoss(jetty); - zkServer.expire(cores.getZkController().getZkClient().getZooKeeper().getSessionId()); } } @@ -182,21 +181,11 @@ public static void causeConnectionLoss(JettySolrRunner jetty) { if (cores != null) { monkeyLog("Will cause connection loss on " + jetty.getLocalPort()); SolrZkClient zkClient = cores.getZkController().getZkClient(); - causeConnectionLoss(zkClient.getZooKeeper()); - } - } - - public static void causeConnectionLoss(ZooKeeper zooKeeper) { - assert zooKeeper instanceof TestableZooKeeper - : "Can only cause connection loss for TestableZookeeper"; - if (zooKeeper instanceof TestableZooKeeper) { try { - ((TestableZooKeeper) zooKeeper).testableConnloss(); - } catch (IOException ignored) { - // best effort + KillSession.kill(zkClient.getCuratorFramework().getZookeeperClient().getZooKeeper()); + } catch (Exception e) { + log.warn("Exception causing connection loss", e); } - } else { - // TODO what now? } } diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java index 26f33c7c60d..54f23d71260 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java @@ -54,6 +54,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import javax.servlet.Filter; +import org.apache.curator.test.KillSession; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.client.solrj.embedded.SSLConfig; @@ -788,7 +789,14 @@ public void expireZkSession(JettySolrRunner jetty) { CoreContainer cores = jetty.getCoreContainer(); if (cores != null) { ChaosMonkey.causeConnectionLoss(jetty); - zkServer.expire(cores.getZkController().getZkClient().getZooKeeper().getSessionId()); + SolrZkClient zkClient = cores.getZkController().getZkClient(); + long sessionId = zkClient.getZkSessionId(); + zkServer.expire(sessionId); + try { + KillSession.kill(zkClient.getCuratorFramework().getZookeeperClient().getZooKeeper()); + } catch (Exception e) { + log.error("Exception killing session", e); + } if (log.isInfoEnabled()) { log.info("Expired zookeeper session from node {}", jetty.getBaseUrl()); } diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java index 14d2a5a1f86..8bd7396c7a7 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java @@ -45,7 +45,6 @@ import org.apache.solr.common.util.Utils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Op; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooDefs; @@ -796,42 +795,26 @@ public void buildZooKeeper(Path solrhome, String config, String schema) throws E props.put("configName", "conf1"); final ZkNodeProps zkProps = new ZkNodeProps(props); - List ops = new ArrayList<>(2); - String path = "/collections"; - ops.add( - Op.create( - path, null, chRootClient.getZkACLProvider().getACLsToAdd(path), CreateMode.PERSISTENT)); - path = "/collections/collection1"; - ops.add( - Op.create( - path, - Utils.toJSON(zkProps), - chRootClient.getZkACLProvider().getACLsToAdd(path), - CreateMode.PERSISTENT)); - path = "/collections/collection1/shards"; - ops.add( - Op.create( - path, null, chRootClient.getZkACLProvider().getACLsToAdd(path), CreateMode.PERSISTENT)); - path = "/collections/control_collection"; - ops.add( - Op.create( - path, - Utils.toJSON(zkProps), - chRootClient.getZkACLProvider().getACLsToAdd(path), - CreateMode.PERSISTENT)); - path = "/collections/control_collection/shards"; - ops.add( - Op.create( - path, null, chRootClient.getZkACLProvider().getACLsToAdd(path), CreateMode.PERSISTENT)); - path = "/configs"; - ops.add( - Op.create( - path, null, chRootClient.getZkACLProvider().getACLsToAdd(path), CreateMode.PERSISTENT)); - path = "/configs/conf1"; - ops.add( - Op.create( - path, null, chRootClient.getZkACLProvider().getACLsToAdd(path), CreateMode.PERSISTENT)); - chRootClient.multi(ops, true); + chRootClient.multi( + op -> op.create().withMode(CreateMode.PERSISTENT).forPath("/collections", null), + op -> + op.create() + .withMode(CreateMode.PERSISTENT) + .forPath("/collections/collection1", Utils.toJSON(zkProps)), + op -> + op.create() + .withMode(CreateMode.PERSISTENT) + .forPath("/collections/collection1/shards", null), + op -> + op.create() + .withMode(CreateMode.PERSISTENT) + .forPath("/collections/control_collection", Utils.toJSON(zkProps)), + op -> + op.create() + .withMode(CreateMode.PERSISTENT) + .forPath("/collections/control_collection/shards", null), + op -> op.create().withMode(CreateMode.PERSISTENT).forPath("/configs", null), + op -> op.create().withMode(CreateMode.PERSISTENT).forPath("/configs/conf1", null)); // for now, always upload the config and schema to the canonical names putConfig("conf1", chRootClient, solrhome, config, "solrconfig.xml"); diff --git a/versions.lock b/versions.lock index f1d2e911ede..7418e07bfd9 100644 --- a/versions.lock +++ b/versions.lock @@ -48,7 +48,7 @@ com.google.cloud:google-cloud-storage:2.40.1 (2 constraints: cf1cc626) com.google.code.gson:gson:2.11.0 (6 constraints: 0c550bc0) com.google.errorprone:error_prone_annotations:2.28.0 (15 constraints: a5c51259) com.google.guava:failureaccess:1.0.2 (2 constraints: fb19bf37) -com.google.guava:guava:33.1.0-jre (26 constraints: 0280374a) +com.google.guava:guava:33.1.0-jre (27 constraints: 698fe64d) com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava (2 constraints: 4b35b0a0) com.google.http-client:google-http-client:1.44.2 (11 constraints: 45bf29b8) com.google.http-client:google-http-client-apache-v2:1.44.2 (2 constraints: b9201d76) @@ -100,7 +100,7 @@ edu.ucar:netcdf4:4.5.5 (1 constraints: 650c0402) edu.ucar:udunits:4.5.5 (1 constraints: 2b06034e) edu.usc.ir:sentiment-analysis-parser:0.1 (1 constraints: fa0b50e9) io.dropwizard.metrics:metrics-annotation:4.2.26 (1 constraints: 361073b0) -io.dropwizard.metrics:metrics-core:4.2.26 (9 constraints: e382b9e2) +io.dropwizard.metrics:metrics-core:4.2.26 (10 constraints: b190d0d8) io.dropwizard.metrics:metrics-graphite:4.2.26 (1 constraints: 4005473b) io.dropwizard.metrics:metrics-healthchecks:4.2.26 (1 constraints: 1d1183c9) io.dropwizard.metrics:metrics-jetty10:4.2.26 (1 constraints: 4005473b) @@ -194,9 +194,10 @@ org.apache.commons:commons-exec:1.4.0 (2 constraints: 031132cf) org.apache.commons:commons-lang3:3.15.0 (6 constraints: cc4e807b) org.apache.commons:commons-math3:3.6.1 (5 constraints: 57322799) org.apache.commons:commons-text:1.12.0 (2 constraints: 651f97e5) -org.apache.curator:curator-client:5.7.0 (2 constraints: ec14cea3) -org.apache.curator:curator-framework:5.7.0 (2 constraints: 0914ad75) -org.apache.curator:curator-recipes:5.7.0 (1 constraints: 0e051936) +org.apache.curator:curator-client:5.7.1 (2 constraints: ee14fda3) +org.apache.curator:curator-framework:5.7.1 (2 constraints: 0b14da75) +org.apache.curator:curator-recipes:5.7.1 (1 constraints: 0f051a36) +org.apache.curator:curator-test:5.7.1 (1 constraints: 0f051a36) org.apache.hadoop:hadoop-annotations:3.4.0 (1 constraints: 09050636) org.apache.hadoop:hadoop-auth:3.4.0 (1 constraints: 09050636) org.apache.hadoop:hadoop-client-api:3.4.0 (3 constraints: 16281b5e) @@ -283,9 +284,9 @@ org.apache.tika:tika-core:1.28.5 (2 constraints: d8118f11) org.apache.tika:tika-parsers:1.28.5 (1 constraints: 42054a3b) org.apache.tomcat:annotations-api:6.0.53 (1 constraints: 40054e3b) org.apache.xmlbeans:xmlbeans:5.0.3 (2 constraints: 72173075) -org.apache.zookeeper:zookeeper:3.9.2 (3 constraints: a51e5e0a) +org.apache.zookeeper:zookeeper:3.9.2 (2 constraints: 15105ed4) org.apache.zookeeper:zookeeper-jute:3.9.2 (2 constraints: 9d12b123) -org.apiguardian:apiguardian-api:1.1.2 (2 constraints: 601bd5a8) +org.apiguardian:apiguardian-api:1.1.2 (4 constraints: b03c1624) org.bitbucket.b_c:jose4j:0.9.6 (2 constraints: 14106bd4) org.bouncycastle:bcmail-jdk15on:1.70 (1 constraints: 310c8af5) org.bouncycastle:bcpkix-jdk15on:1.70 (2 constraints: ce1b11b3) @@ -354,6 +355,9 @@ org.jetbrains.kotlin:kotlin-stdlib:1.9.10 (4 constraints: 5c405537) org.jetbrains.kotlin:kotlin-stdlib-common:1.9.10 (3 constraints: 3a2c8a72) org.jetbrains.kotlin:kotlin-stdlib-jdk7:1.9.10 (1 constraints: e210ffd2) org.jetbrains.kotlin:kotlin-stdlib-jdk8:1.9.10 (4 constraints: e23809e7) +org.junit:junit-bom:5.6.2 (2 constraints: 67212a70) +org.junit.jupiter:junit-jupiter-api:5.6.2 (2 constraints: e516c64d) +org.junit.platform:junit-platform-commons:1.6.2 (2 constraints: 7c1841a1) org.locationtech.jts:jts-core:1.19.0 (2 constraints: a31de760) org.locationtech.jts.io:jts-io-common:1.19.0 (1 constraints: 930d513a) org.locationtech.proj4j:proj4j:1.2.2 (1 constraints: 5d0daf2c) @@ -361,6 +365,7 @@ org.locationtech.spatial4j:spatial4j:0.8 (1 constraints: 59105498) org.lz4:lz4-java:1.8.0 (1 constraints: fc0cd91b) org.opengis:geoapi:3.0.1 (7 constraints: 1361d046) org.openjdk.jmh:jmh-core:1.37 (1 constraints: df04fc30) +org.opentest4j:opentest4j:1.2.0 (1 constraints: 670f0e7d) org.osgi:org.osgi.resource:1.0.0 (1 constraints: e60f2999) org.osgi:org.osgi.service.serviceloader:1.0.0 (1 constraints: e60f2999) org.osgi:osgi.annotation:8.1.0 (1 constraints: 0b051636) @@ -386,7 +391,7 @@ org.tallison:metadata-extractor:2.17.1.0 (1 constraints: f00c3b28) org.tallison.xmp:xmpcore-shaded:6.1.10 (1 constraints: 300e8d49) org.threeten:threetenbp:1.6.9 (4 constraints: 2833ea68) org.tukaani:xz:1.9 (1 constraints: 030c5be9) -org.xerial.snappy:snappy-java:1.1.10.5 (5 constraints: be46ae36) +org.xerial.snappy:snappy-java:1.1.10.5 (6 constraints: 17552375) software.amazon.awssdk:annotations:2.26.19 (28 constraints: 48a79471) software.amazon.awssdk:apache-client:2.26.19 (4 constraints: 112adae3) software.amazon.awssdk:arns:2.26.19 (2 constraints: 231878c1) diff --git a/versions.props b/versions.props index 1c681b6ebfc..0433da0fc4f 100644 --- a/versions.props +++ b/versions.props @@ -39,7 +39,7 @@ org.apache.commons:commons-configuration2=2.11.0 org.apache.commons:commons-exec=1.4.0 org.apache.commons:commons-lang3=3.15.0 org.apache.commons:commons-math3=3.6.1 -org.apache.curator:*=5.7.0 +org.apache.curator:*=5.7.1 org.apache.hadoop.thirdparty:*=1.2.0 org.apache.hadoop:*=3.4.0 org.apache.httpcomponents:httpclient=4.5.14