acl, CreateMode crea
return new Create(path, data, acl, createMode);
}
+ /**
+ * Constructs a create operation which uses given op code if no one is inferred from create mode.
+ *
+ * @param path
+ * the path for the node
+ * @param data
+ * the initial data for the node
+ * @param options
+ * options for creating znode
+ * @param defaultOpCode
+ * op code to be used if no one is inferred from create mode
+ */
+ static Op create(String path, byte[] data, CreateOptions options, int defaultOpCode) {
+ if (options.getCreateMode().isTTL()) {
+ return new CreateTTL(path, data, options.getAcl(), options.getCreateMode(), options.getTtl());
+ }
+ return new Create(path, data, options.getAcl(), options.getCreateMode(), defaultOpCode);
+ }
+
+ /**
+ * Constructs a create operation which uses {@link ZooDefs.OpCode#create2} if no one is inferred from create mode.
+ *
+ * The corresponding {@link OpResult.CreateResult#getStat()} could be null if connected to server without this
+ * patch.
+ *
+ * @param path
+ * the path for the node
+ * @param data
+ * the initial data for the node
+ * @param options
+ * options for creating znode
+ */
+ public static Op create(String path, byte[] data, CreateOptions options) {
+ return create(path, data, options, ZooDefs.OpCode.create2);
+ }
+
/**
* Constructs a delete operation. Arguments are as for the ZooKeeper method of the same name.
* @see ZooKeeper#delete(String, int)
@@ -263,21 +299,29 @@ public static class Create extends Op {
protected int flags;
private Create(String path, byte[] data, List acl, int flags) {
- super(getOpcode(CreateMode.fromFlag(flags, CreateMode.PERSISTENT)), path, OpKind.TRANSACTION);
+ this(path, data, acl, flags, ZooDefs.OpCode.create);
+ }
+
+ private Create(String path, byte[] data, List acl, int flags, int defaultOpCode) {
+ super(getOpcode(CreateMode.fromFlag(flags, CreateMode.PERSISTENT), defaultOpCode), path, OpKind.TRANSACTION);
this.data = data;
this.acl = acl;
this.flags = flags;
}
- private static int getOpcode(CreateMode createMode) {
+ private static int getOpcode(CreateMode createMode, int defaultOpCode) {
if (createMode.isTTL()) {
return ZooDefs.OpCode.createTTL;
}
- return createMode.isContainer() ? ZooDefs.OpCode.createContainer : ZooDefs.OpCode.create;
+ return createMode.isContainer() ? ZooDefs.OpCode.createContainer : defaultOpCode;
}
private Create(String path, byte[] data, List acl, CreateMode createMode) {
- super(getOpcode(createMode), path, OpKind.TRANSACTION);
+ this(path, data, acl, createMode, ZooDefs.OpCode.create);
+ }
+
+ private Create(String path, byte[] data, List acl, CreateMode createMode, int defaultOpCode) {
+ super(getOpcode(createMode, defaultOpCode), path, OpKind.TRANSACTION);
this.data = data;
this.acl = acl;
this.flags = createMode.toFlag();
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java
index d2063414470..a76e2f63b31 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java
@@ -49,7 +49,14 @@ public class ZKUtil {
* If there is an error with deleting one of the sub-nodes in the tree,
* this operation would abort and would be the responsibility of the app to handle the same.
*
- *
+ * @param zk Zookeeper client
+ * @param pathRoot path to be deleted
+ * @param batchSize number of delete operations to be submitted in one call.
+ * batchSize is also used to decide sync and async delete API invocation.
+ * If batchSize>0 then async otherwise sync delete API is invoked. batchSize>0
+ * gives better performance. batchSize<=0 scenario is handled to preserve
+ * backward compatibility.
+ * @return true if given node and all its sub nodes are deleted successfully otherwise false
* @throws IllegalArgumentException if an invalid path is specified
*/
public static boolean deleteRecursive(
@@ -61,7 +68,15 @@ public static boolean deleteRecursive(
List tree = listSubTreeBFS(zk, pathRoot);
LOG.debug("Deleting tree: {}", tree);
- return deleteInBatch(zk, tree, batchSize);
+ if (batchSize > 0) {
+ return deleteInBatch(zk, tree, batchSize);
+ } else {
+ for (int i = tree.size() - 1; i >= 0; --i) {
+ //Delete the leaves first and eventually get rid of the root
+ zk.delete(tree.get(i), -1); //Delete all versions of the node with -1.
+ }
+ return true;
+ }
}
/**
@@ -73,7 +88,8 @@ public static boolean deleteRecursive(
public static void deleteRecursive(
ZooKeeper zk,
final String pathRoot) throws InterruptedException, KeeperException {
- deleteRecursive(zk, pathRoot, 1000);
+ // batchSize=0 is passed to preserve the backward compatibility with older clients.
+ deleteRecursive(zk, pathRoot, 0);
}
private static class BatchedDeleteCbContext {
@@ -93,10 +109,10 @@ private static boolean deleteInBatch(ZooKeeper zk, List tree, int batchS
List ops = new ArrayList<>();
BatchedDeleteCbContext context = new BatchedDeleteCbContext(rateLimit);
MultiCallback cb = (rc, path, ctx, opResults) -> {
- ((BatchedDeleteCbContext) ctx).sem.release();
if (rc != Code.OK.intValue()) {
((BatchedDeleteCbContext) ctx).success.set(false);
}
+ ((BatchedDeleteCbContext) ctx).sem.release();
};
// Delete the leaves first and eventually get rid of the root
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java
index 42a4e284e4d..de2f79dcf1d 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java
@@ -92,27 +92,28 @@ public static String send4LetterWord(
boolean secure,
int timeout) throws IOException, SSLContextException {
LOG.info("connecting to {} {}", host, port);
- Socket sock;
- InetSocketAddress hostaddress = host != null
- ? new InetSocketAddress(host, port)
- : new InetSocketAddress(InetAddress.getByName(null), port);
- if (secure) {
- LOG.info("using secure socket");
- try (X509Util x509Util = new ClientX509Util()) {
- SSLContext sslContext = x509Util.getDefaultSSLContext();
- SSLSocketFactory socketFactory = sslContext.getSocketFactory();
- SSLSocket sslSock = (SSLSocket) socketFactory.createSocket();
- sslSock.connect(hostaddress, timeout);
- sslSock.startHandshake();
- sock = sslSock;
- }
- } else {
- sock = new Socket();
- sock.connect(hostaddress, timeout);
- }
- sock.setSoTimeout(timeout);
+
+ Socket sock = null;
BufferedReader reader = null;
try {
+ InetSocketAddress hostaddress = host != null
+ ? new InetSocketAddress(host, port)
+ : new InetSocketAddress(InetAddress.getByName(null), port);
+ if (secure) {
+ LOG.info("using secure socket");
+ try (X509Util x509Util = new ClientX509Util()) {
+ SSLContext sslContext = x509Util.getDefaultSSLContext();
+ SSLSocketFactory socketFactory = sslContext.getSocketFactory();
+ SSLSocket sslSock = (SSLSocket) socketFactory.createSocket();
+ sslSock.connect(hostaddress, timeout);
+ sslSock.startHandshake();
+ sock = sslSock;
+ }
+ } else {
+ sock = new Socket();
+ sock.connect(hostaddress, timeout);
+ }
+ sock.setSoTimeout(timeout);
OutputStream outstream = sock.getOutputStream();
outstream.write(cmd.getBytes(UTF_8));
outstream.flush();
@@ -133,7 +134,9 @@ public static String send4LetterWord(
} catch (SocketTimeoutException e) {
throw new IOException("Exception while executing four letter word: " + cmd, e);
} finally {
- sock.close();
+ if (sock != null) {
+ sock.close();
+ }
if (reader != null) {
reader.close();
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java b/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java
index 940bcfe228b..cafa66610df 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.security.PrivilegedActionException;
import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicReference;
import javax.security.auth.Subject;
import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.Configuration;
@@ -64,7 +65,6 @@ public class ZooKeeperSaslClient {
*/
@Deprecated
public static final String ENABLE_CLIENT_SASL_DEFAULT = "true";
- private volatile boolean initializedLogin = false;
/**
* Returns true if the SASL client is enabled. By default, the client
@@ -112,7 +112,7 @@ public String getLoginContext() {
return null;
}
- public ZooKeeperSaslClient(final String serverPrincipal, ZKClientConfig clientConfig) throws LoginException {
+ public ZooKeeperSaslClient(final String serverPrincipal, ZKClientConfig clientConfig, AtomicReference loginRef) throws LoginException {
/**
* ZOOKEEPER-1373: allow system property to specify the JAAS
* configuration section that the zookeeper client should use.
@@ -136,7 +136,8 @@ public ZooKeeperSaslClient(final String serverPrincipal, ZKClientConfig clientCo
}
if (entries != null) {
this.configStatus = "Will attempt to SASL-authenticate using Login Context section '" + clientSection + "'";
- this.saslClient = createSaslClient(serverPrincipal, clientSection);
+ this.saslClient = createSaslClient(serverPrincipal, clientSection, loginRef);
+ this.login = loginRef.get();
} else {
// Handle situation of clientSection's being null: it might simply because the client does not intend to
// use SASL, so not necessarily an error.
@@ -234,26 +235,25 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta
private SaslClient createSaslClient(
final String servicePrincipal,
- final String loginContext) throws LoginException {
+ final String loginContext,
+ final AtomicReference loginRef) throws LoginException {
try {
- if (!initializedLogin) {
- synchronized (this) {
- if (login == null) {
- LOG.debug("JAAS loginContext is: {}", loginContext);
- // note that the login object is static: it's shared amongst all zookeeper-related connections.
- // in order to ensure the login is initialized only once, it must be synchronized the code snippet.
- login = new Login(loginContext, new SaslClientCallbackHandler(null, "Client"), clientConfig);
- login.startThreadIfNeeded();
- initializedLogin = true;
- }
+ if (loginRef.get() == null) {
+ LOG.debug("JAAS loginContext is: {}", loginContext);
+ // note that the login object is static: it's shared amongst all zookeeper-related connections.
+ // in order to ensure the login is initialized only once, it must be synchronized the code snippet.
+ Login l = new Login(loginContext, new SaslClientCallbackHandler(null, "Client"), clientConfig);
+ if (loginRef.compareAndSet(null, l)) {
+ l.startThreadIfNeeded();
}
}
- return SecurityUtils.createSaslClient(login.getSubject(), servicePrincipal, "zookeeper", "zk-sasl-md5", LOG, "Client");
+ return SecurityUtils.createSaslClient(loginRef.get().getSubject(),
+ servicePrincipal, "zookeeper", "zk-sasl-md5", LOG, "Client");
} catch (LoginException e) {
// We throw LoginExceptions...
throw e;
} catch (Exception e) {
- // ..but consume (with a log message) all other types of exceptions.
+ // ...but consume (with a log message) all other types of exceptions.
LOG.error("Exception while trying to create SASL client.", e);
return null;
}
@@ -451,15 +451,4 @@ public boolean clientTunneledAuthenticationInProgress() {
return false;
}
}
-
- /**
- * close login thread if running
- */
- public void shutdown() {
- if (null != login) {
- login.shutdown();
- login = null;
- }
- }
-
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/NetUtils.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/NetUtils.java
index be8cb9a6389..a02499a6877 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/NetUtils.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/NetUtils.java
@@ -27,17 +27,18 @@
*/
public class NetUtils {
+ /**
+ * Prefer using the hostname for formatting, but without requesting reverse DNS lookup.
+ * Fall back to IP address if hostname is unavailable and use [] brackets for IPv6 literal.
+ */
public static String formatInetAddr(InetSocketAddress addr) {
+ String hostString = addr.getHostString();
InetAddress ia = addr.getAddress();
- if (ia == null) {
- return String.format("%s:%s", addr.getHostString(), addr.getPort());
- }
-
- if (ia instanceof Inet6Address) {
- return String.format("[%s]:%s", ia.getHostAddress(), addr.getPort());
+ if (ia instanceof Inet6Address && hostString.contains(":")) {
+ return String.format("[%s]:%s", hostString, addr.getPort());
} else {
- return String.format("%s:%s", ia.getHostAddress(), addr.getPort());
+ return String.format("%s:%s", hostString, addr.getPort());
}
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/SSLContextAndOptions.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/SSLContextAndOptions.java
index 01e97c3f594..4e61d7b9b42 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/SSLContextAndOptions.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/SSLContextAndOptions.java
@@ -19,6 +19,7 @@
package org.apache.zookeeper.common;
import static java.util.Objects.requireNonNull;
+import io.netty.handler.ssl.DelegatingSslContext;
import io.netty.handler.ssl.IdentityCipherSuiteFilter;
import io.netty.handler.ssl.JdkSslContext;
import io.netty.handler.ssl.SslContext;
@@ -29,6 +30,7 @@
import java.util.Collections;
import java.util.List;
import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
import javax.net.ssl.SSLParameters;
import javax.net.ssl.SSLServerSocket;
import javax.net.ssl.SSLSocket;
@@ -53,6 +55,8 @@ public class SSLContextAndOptions {
private final X509Util.ClientAuth clientAuth;
private final SSLContext sslContext;
private final int handshakeDetectionTimeoutMillis;
+ private final ZKConfig config;
+
/**
* Note: constructor is intentionally package-private, only the X509Util class should be creating instances of this
@@ -70,6 +74,7 @@ public class SSLContextAndOptions {
this.cipherSuitesAsList = Collections.unmodifiableList(Arrays.asList(ciphers));
this.clientAuth = getClientAuth(config);
this.handshakeDetectionTimeoutMillis = getHandshakeDetectionTimeoutMillis(config);
+ this.config = config;
}
public SSLContext getSSLContext() {
@@ -101,18 +106,32 @@ public SSLServerSocket createSSLServerSocket(int port) throws IOException {
return configureSSLServerSocket(sslServerSocket);
}
- public SslContext createNettyJdkSslContext(SSLContext sslContext, boolean isClientSocket) {
- return new JdkSslContext(
+ public SslContext createNettyJdkSslContext(SSLContext sslContext) {
+ SslContext sslContext1 = new JdkSslContext(
sslContext,
- isClientSocket,
+ false,
cipherSuitesAsList,
IdentityCipherSuiteFilter.INSTANCE,
null,
- isClientSocket
- ? X509Util.ClientAuth.NONE.toNettyClientAuth()
- : clientAuth.toNettyClientAuth(),
+ clientAuth.toNettyClientAuth(),
enabledProtocols,
false);
+
+ if (x509Util.getFipsMode(config) && x509Util.isClientHostnameVerificationEnabled(config)) {
+ return new DelegatingSslContext(sslContext1) {
+ @Override
+ protected void initEngine(SSLEngine sslEngine) {
+ SSLParameters sslParameters = sslEngine.getSSLParameters();
+ sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
+ sslEngine.setSSLParameters(sslParameters);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Client hostname verification: enabled HTTPS style endpoint identification algorithm");
+ }
+ }
+ };
+ } else {
+ return sslContext1;
+ }
}
public int getHandshakeDetectionTimeoutMillis() {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java
index 0cc3eb218bb..eaca8c20e4d 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java
@@ -66,6 +66,7 @@ public abstract class X509Util implements Closeable, AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(X509Util.class);
private static final String REJECT_CLIENT_RENEGOTIATION_PROPERTY = "jdk.tls.rejectClientInitiatedRenegotiation";
+ private static final String FIPS_MODE_PROPERTY = "zookeeper.fips-mode";
static {
// Client-initiated renegotiation in TLS is unsafe and
@@ -143,36 +144,30 @@ public io.netty.handler.ssl.ClientAuth toNettyClientAuth() {
}
}
- private String sslProtocolProperty = getConfigPrefix() + "protocol";
- private String sslEnabledProtocolsProperty = getConfigPrefix() + "enabledProtocols";
- private String cipherSuitesProperty = getConfigPrefix() + "ciphersuites";
- private String sslKeystoreLocationProperty = getConfigPrefix() + "keyStore.location";
- private String sslKeystorePasswdProperty = getConfigPrefix() + "keyStore.password";
- private String sslKeystorePasswdPathProperty = getConfigPrefix() + "keyStore.passwordPath";
- private String sslKeystoreTypeProperty = getConfigPrefix() + "keyStore.type";
- private String sslTruststoreLocationProperty = getConfigPrefix() + "trustStore.location";
- private String sslTruststorePasswdProperty = getConfigPrefix() + "trustStore.password";
- private String sslTruststorePasswdPathProperty = getConfigPrefix() + "trustStore.passwordPath";
- private String sslTruststoreTypeProperty = getConfigPrefix() + "trustStore.type";
- private String sslContextSupplierClassProperty = getConfigPrefix() + "context.supplier.class";
- private String sslHostnameVerificationEnabledProperty = getConfigPrefix() + "hostnameVerification";
- private String sslCrlEnabledProperty = getConfigPrefix() + "crl";
- private String sslOcspEnabledProperty = getConfigPrefix() + "ocsp";
- private String sslClientAuthProperty = getConfigPrefix() + "clientAuth";
- private String sslHandshakeDetectionTimeoutMillisProperty = getConfigPrefix() + "handshakeDetectionTimeoutMillis";
-
- private ZKConfig zkConfig;
- private AtomicReference defaultSSLContextAndOptions = new AtomicReference<>(null);
+ private final String sslProtocolProperty = getConfigPrefix() + "protocol";
+ private final String sslEnabledProtocolsProperty = getConfigPrefix() + "enabledProtocols";
+ private final String cipherSuitesProperty = getConfigPrefix() + "ciphersuites";
+ private final String sslKeystoreLocationProperty = getConfigPrefix() + "keyStore.location";
+ private final String sslKeystorePasswdProperty = getConfigPrefix() + "keyStore.password";
+ private final String sslKeystorePasswdPathProperty = getConfigPrefix() + "keyStore.passwordPath";
+ private final String sslKeystoreTypeProperty = getConfigPrefix() + "keyStore.type";
+ private final String sslTruststoreLocationProperty = getConfigPrefix() + "trustStore.location";
+ private final String sslTruststorePasswdProperty = getConfigPrefix() + "trustStore.password";
+ private final String sslTruststorePasswdPathProperty = getConfigPrefix() + "trustStore.passwordPath";
+ private final String sslTruststoreTypeProperty = getConfigPrefix() + "trustStore.type";
+ private final String sslContextSupplierClassProperty = getConfigPrefix() + "context.supplier.class";
+ private final String sslHostnameVerificationEnabledProperty = getConfigPrefix() + "hostnameVerification";
+ private final String sslCrlEnabledProperty = getConfigPrefix() + "crl";
+ private final String sslOcspEnabledProperty = getConfigPrefix() + "ocsp";
+ private final String sslClientAuthProperty = getConfigPrefix() + "clientAuth";
+ private final String sslHandshakeDetectionTimeoutMillisProperty = getConfigPrefix() + "handshakeDetectionTimeoutMillis";
+
+ private final AtomicReference defaultSSLContextAndOptions = new AtomicReference<>(null);
private FileChangeWatcher keyStoreFileWatcher;
private FileChangeWatcher trustStoreFileWatcher;
public X509Util() {
- this(null);
- }
-
- public X509Util(ZKConfig zkConfig) {
- this.zkConfig = zkConfig;
keyStoreFileWatcher = trustStoreFileWatcher = null;
}
@@ -258,6 +253,22 @@ public String getSslHandshakeDetectionTimeoutMillisProperty() {
return sslHandshakeDetectionTimeoutMillisProperty;
}
+ public String getFipsModeProperty() {
+ return FIPS_MODE_PROPERTY;
+ }
+
+ public boolean getFipsMode(ZKConfig config) {
+ return config.getBoolean(FIPS_MODE_PROPERTY, false);
+ }
+
+ public boolean isServerHostnameVerificationEnabled(ZKConfig config) {
+ return config.getBoolean(this.getSslHostnameVerificationEnabledProperty(), true);
+ }
+
+ public boolean isClientHostnameVerificationEnabled(ZKConfig config) {
+ return isServerHostnameVerificationEnabled(config) && shouldVerifyClientHostname();
+ }
+
public SSLContext getDefaultSSLContext() throws X509Exception.SSLContextException {
return getDefaultSSLContextAndOptions().getSSLContext();
}
@@ -289,7 +300,7 @@ private SSLContextAndOptions createSSLContextAndOptions() throws SSLContextExcep
* configuration from system property. Reading property from
* configuration will be same reading from system property
*/
- return createSSLContextAndOptions(zkConfig == null ? new ZKConfig() : zkConfig);
+ return createSSLContextAndOptions(new ZKConfig());
}
/**
@@ -369,14 +380,18 @@ public SSLContextAndOptions createSSLContextAndOptionsFromConfig(ZKConfig config
boolean sslCrlEnabled = config.getBoolean(this.sslCrlEnabledProperty);
boolean sslOcspEnabled = config.getBoolean(this.sslOcspEnabledProperty);
- boolean sslServerHostnameVerificationEnabled = config.getBoolean(this.getSslHostnameVerificationEnabledProperty(), true);
- boolean sslClientHostnameVerificationEnabled = sslServerHostnameVerificationEnabled && shouldVerifyClientHostname();
+ boolean sslServerHostnameVerificationEnabled = isServerHostnameVerificationEnabled(config);
+ boolean sslClientHostnameVerificationEnabled = isClientHostnameVerificationEnabled(config);
+ boolean fipsMode = getFipsMode(config);
if (trustStoreLocationProp.isEmpty()) {
LOG.warn("{} not specified", getSslTruststoreLocationProperty());
} else {
try {
- trustManagers = new TrustManager[]{createTrustManager(trustStoreLocationProp, trustStorePasswordProp, trustStoreTypeProp, sslCrlEnabled, sslOcspEnabled, sslServerHostnameVerificationEnabled, sslClientHostnameVerificationEnabled)};
+ trustManagers = new TrustManager[]{
+ createTrustManager(trustStoreLocationProp, trustStorePasswordProp, trustStoreTypeProp, sslCrlEnabled,
+ sslOcspEnabled, sslServerHostnameVerificationEnabled, sslClientHostnameVerificationEnabled,
+ fipsMode)};
} catch (TrustManagerException trustManagerException) {
throw new SSLContextException("Failed to create TrustManager", trustManagerException);
} catch (IllegalArgumentException e) {
@@ -481,16 +496,17 @@ public static X509KeyManager createKeyManager(
/**
* Creates a trust manager by loading the trust store from the given file
* of the given type, optionally decrypting it using the given password.
- * @param trustStoreLocation the location of the trust store file.
- * @param trustStorePassword optional password to decrypt the trust store
- * (only applies to JKS trust stores). If empty,
- * assumes the trust store is not encrypted.
- * @param trustStoreTypeProp must be JKS, PEM, PKCS12, BCFKS or null. If
- * null, attempts to autodetect the trust store
- * type from the file extension (e.g. .jks / .pem).
- * @param crlEnabled enable CRL (certificate revocation list) checks.
- * @param ocspEnabled enable OCSP (online certificate status protocol)
- * checks.
+ *
+ * @param trustStoreLocation the location of the trust store file.
+ * @param trustStorePassword optional password to decrypt the trust store
+ * (only applies to JKS trust stores). If empty,
+ * assumes the trust store is not encrypted.
+ * @param trustStoreTypeProp must be JKS, PEM, PKCS12, BCFKS or null. If
+ * null, attempts to autodetect the trust store
+ * type from the file extension (e.g. .jks / .pem).
+ * @param crlEnabled enable CRL (certificate revocation list) checks.
+ * @param ocspEnabled enable OCSP (online certificate status protocol)
+ * checks.
* @param serverHostnameVerificationEnabled if true, verify hostnames of
* remote servers that client
* sockets created by this
@@ -510,7 +526,8 @@ public static X509TrustManager createTrustManager(
boolean crlEnabled,
boolean ocspEnabled,
final boolean serverHostnameVerificationEnabled,
- final boolean clientHostnameVerificationEnabled) throws TrustManagerException {
+ final boolean clientHostnameVerificationEnabled,
+ final boolean fipsMode) throws TrustManagerException {
if (trustStorePassword == null) {
trustStorePassword = "";
}
@@ -534,7 +551,17 @@ public static X509TrustManager createTrustManager(
for (final TrustManager tm : tmf.getTrustManagers()) {
if (tm instanceof X509ExtendedTrustManager) {
- return new ZKTrustManager((X509ExtendedTrustManager) tm, serverHostnameVerificationEnabled, clientHostnameVerificationEnabled);
+ if (fipsMode) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("FIPS mode is ON: selecting standard x509 trust manager {}", tm);
+ }
+ return (X509TrustManager) tm;
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("FIPS mode is OFF: creating ZKTrustManager");
+ }
+ return new ZKTrustManager((X509ExtendedTrustManager) tm, serverHostnameVerificationEnabled,
+ clientHostnameVerificationEnabled);
}
}
throw new TrustManagerException("Couldn't find X509TrustManager");
@@ -600,7 +627,7 @@ private FileChangeWatcher newFileChangeWatcher(String fileLocation) throws IOExc
*/
public void enableCertFileReloading() throws IOException {
LOG.info("enabling cert file reloading");
- ZKConfig config = zkConfig == null ? new ZKConfig() : zkConfig;
+ ZKConfig config = new ZKConfig();
FileChangeWatcher newKeyStoreFileWatcher = newFileChangeWatcher(config.getProperty(sslKeystoreLocationProperty));
if (newKeyStoreFileWatcher != null) {
// stop old watcher if there is one
@@ -627,6 +654,7 @@ public void enableCertFileReloading() throws IOException {
*/
@Override
public void close() {
+ defaultSSLContextAndOptions.set(null);
if (keyStoreFileWatcher != null) {
keyStoreFileWatcher.stop();
keyStoreFileWatcher = null;
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java
index 1d1a6c9eabe..0d98d35ba43 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java
@@ -82,6 +82,7 @@ public ZKConfig(String configPath) throws ConfigException {
public ZKConfig(File configFile) throws ConfigException {
this();
addConfiguration(configFile);
+ LOG.info("ZK Config {}", this.properties);
}
private void init() {
@@ -130,6 +131,7 @@ private void putSSLProperties(X509Util x509Util) {
properties.put(x509Util.getSslOcspEnabledProperty(), System.getProperty(x509Util.getSslOcspEnabledProperty()));
properties.put(x509Util.getSslClientAuthProperty(), System.getProperty(x509Util.getSslClientAuthProperty()));
properties.put(x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), System.getProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty()));
+ properties.put(x509Util.getFipsModeProperty(), System.getProperty(x509Util.getFipsModeProperty()));
}
/**
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKHostnameVerifier.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKHostnameVerifier.java
index db5af19c54a..247ae391723 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKHostnameVerifier.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKHostnameVerifier.java
@@ -49,7 +49,7 @@
* Note: copied from Apache httpclient with some modifications. We want host verification, but depending
* on the httpclient jar caused unexplained performance regressions (even when the code was not used).
*/
-final class ZKHostnameVerifier implements HostnameVerifier {
+class ZKHostnameVerifier implements HostnameVerifier {
/**
* Note: copied from Apache httpclient with some minor modifications. We want host verification, but depending
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKTrustManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKTrustManager.java
index 34bf4dc977b..cbadd1e0af9 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKTrustManager.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKTrustManager.java
@@ -39,11 +39,11 @@ public class ZKTrustManager extends X509ExtendedTrustManager {
private static final Logger LOG = LoggerFactory.getLogger(ZKTrustManager.class);
- private X509ExtendedTrustManager x509ExtendedTrustManager;
- private boolean serverHostnameVerificationEnabled;
- private boolean clientHostnameVerificationEnabled;
+ private final X509ExtendedTrustManager x509ExtendedTrustManager;
+ private final boolean serverHostnameVerificationEnabled;
+ private final boolean clientHostnameVerificationEnabled;
- private ZKHostnameVerifier hostnameVerifier;
+ private final ZKHostnameVerifier hostnameVerifier;
/**
* Instantiate a new ZKTrustManager.
@@ -58,10 +58,21 @@ public class ZKTrustManager extends X509ExtendedTrustManager {
X509ExtendedTrustManager x509ExtendedTrustManager,
boolean serverHostnameVerificationEnabled,
boolean clientHostnameVerificationEnabled) {
+ this(x509ExtendedTrustManager,
+ serverHostnameVerificationEnabled,
+ clientHostnameVerificationEnabled,
+ new ZKHostnameVerifier());
+ }
+
+ ZKTrustManager(
+ X509ExtendedTrustManager x509ExtendedTrustManager,
+ boolean serverHostnameVerificationEnabled,
+ boolean clientHostnameVerificationEnabled,
+ ZKHostnameVerifier hostnameVerifier) {
this.x509ExtendedTrustManager = x509ExtendedTrustManager;
this.serverHostnameVerificationEnabled = serverHostnameVerificationEnabled;
this.clientHostnameVerificationEnabled = clientHostnameVerificationEnabled;
- hostnameVerifier = new ZKHostnameVerifier();
+ this.hostnameVerifier = hostnameVerifier;
}
@Override
@@ -76,6 +87,9 @@ public void checkClientTrusted(
Socket socket) throws CertificateException {
x509ExtendedTrustManager.checkClientTrusted(chain, authType, socket);
if (clientHostnameVerificationEnabled) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Check client trusted socket.getInetAddress(): {}, {}", socket.getInetAddress(), socket);
+ }
performHostVerification(socket.getInetAddress(), chain[0]);
}
}
@@ -87,6 +101,9 @@ public void checkServerTrusted(
Socket socket) throws CertificateException {
x509ExtendedTrustManager.checkServerTrusted(chain, authType, socket);
if (serverHostnameVerificationEnabled) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Check server trusted socket.getInetAddress(): {}, {}", socket.getInetAddress(), socket);
+ }
performHostVerification(socket.getInetAddress(), chain[0]);
}
}
@@ -99,6 +116,9 @@ public void checkClientTrusted(
x509ExtendedTrustManager.checkClientTrusted(chain, authType, engine);
if (clientHostnameVerificationEnabled) {
try {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Check client trusted engine.getPeerHost(): {}, {}", engine.getPeerHost(), engine);
+ }
performHostVerification(InetAddress.getByName(engine.getPeerHost()), chain[0]);
} catch (UnknownHostException e) {
throw new CertificateException("Failed to verify host", e);
@@ -115,6 +135,9 @@ public void checkServerTrusted(
x509ExtendedTrustManager.checkServerTrusted(chain, authType, engine);
if (serverHostnameVerificationEnabled) {
try {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Check server trusted engine.getPeerHost(): {}, {}", engine.getPeerHost(), engine);
+ }
performHostVerification(InetAddress.getByName(engine.getPeerHost()), chain[0]);
} catch (UnknownHostException e) {
throw new CertificateException("Failed to verify host", e);
@@ -133,9 +156,12 @@ public void checkServerTrusted(X509Certificate[] chain, String authType) throws
}
/**
- * Compares peer's hostname with the one stored in the provided client certificate. Performs verification
+ * Compares peer's hostname with the one stored in the provided certificate. Performs verification
* with the help of provided HostnameVerifier.
*
+ * Attempts to verify the IP address first, if it fails, check the hostname. Performs reverse DNS lookup
+ * if hostname is not available. (Mostly the case in client verifications.)
+ *
* @param inetAddress Peer's inet address.
* @param certificate Peer's certificate
* @throws CertificateException Thrown if the provided certificate doesn't match the peer hostname.
@@ -143,19 +169,23 @@ public void checkServerTrusted(X509Certificate[] chain, String authType) throws
private void performHostVerification(
InetAddress inetAddress,
X509Certificate certificate
- ) throws CertificateException {
+ ) throws CertificateException {
String hostAddress = "";
String hostName = "";
try {
hostAddress = inetAddress.getHostAddress();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Trying to verify host address first: {}", hostAddress);
+ }
hostnameVerifier.verify(hostAddress, certificate);
} catch (SSLException addressVerificationException) {
try {
- LOG.debug(
- "Failed to verify host address: {} attempting to verify host name with reverse dns lookup",
- hostAddress,
- addressVerificationException);
hostName = inetAddress.getHostName();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Failed to verify host address: {}, trying to verify host name: {}",
+ hostAddress, hostName);
+ }
hostnameVerifier.verify(hostName, certificate);
} catch (SSLException hostnameVerificationException) {
LOG.error("Failed to verify host address: {}", hostAddress, addressVerificationException);
@@ -164,5 +194,4 @@ private void performHostVerification(
}
}
}
-
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java
index 2818e15aa9e..3febd7fd504 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java
@@ -450,7 +450,10 @@ public void createNode(final String path, byte[] data, List acl, long ephem
if (parent == null) {
throw new KeeperException.NoNodeException();
}
+ List parentAcl;
synchronized (parent) {
+ parentAcl = getACL(parent);
+
// Add the ACL to ACL cache first, to avoid the ACL not being
// created race condition during fuzzy snapshot sync.
//
@@ -527,8 +530,9 @@ public void createNode(final String path, byte[] data, List acl, long ephem
updateQuotaStat(lastPrefix, bytes, 1);
}
updateWriteStat(path, bytes);
- dataWatches.triggerWatch(path, Event.EventType.NodeCreated);
- childWatches.triggerWatch(parentName.equals("") ? "/" : parentName, Event.EventType.NodeChildrenChanged);
+ dataWatches.triggerWatch(path, Event.EventType.NodeCreated, acl);
+ childWatches.triggerWatch(parentName.equals("") ? "/" : parentName,
+ Event.EventType.NodeChildrenChanged, parentAcl);
}
/**
@@ -568,8 +572,10 @@ public void deleteNode(String path, long zxid) throws KeeperException.NoNodeExce
if (node == null) {
throw new KeeperException.NoNodeException();
}
+ List acl;
nodes.remove(path);
synchronized (node) {
+ acl = getACL(node);
aclCache.removeUsage(node.acl);
nodeDataSize.addAndGet(-getNodeSize(path, node.data));
}
@@ -577,7 +583,9 @@ public void deleteNode(String path, long zxid) throws KeeperException.NoNodeExce
// Synchronized to sync the containers and ttls change, probably
// only need to sync on containers and ttls, will update it in a
// separate patch.
+ List parentAcl;
synchronized (parent) {
+ parentAcl = getACL(parent);
long eowner = node.stat.getEphemeralOwner();
EphemeralType ephemeralType = EphemeralType.get(eowner);
if (ephemeralType == EphemeralType.CONTAINER) {
@@ -624,9 +632,10 @@ public void deleteNode(String path, long zxid) throws KeeperException.NoNodeExce
"childWatches.triggerWatch " + parentName);
}
- WatcherOrBitSet processed = dataWatches.triggerWatch(path, EventType.NodeDeleted);
- childWatches.triggerWatch(path, EventType.NodeDeleted, processed);
- childWatches.triggerWatch("".equals(parentName) ? "/" : parentName, EventType.NodeChildrenChanged);
+ WatcherOrBitSet processed = dataWatches.triggerWatch(path, EventType.NodeDeleted, acl);
+ childWatches.triggerWatch(path, EventType.NodeDeleted, acl, processed);
+ childWatches.triggerWatch("".equals(parentName) ? "/" : parentName,
+ EventType.NodeChildrenChanged, parentAcl);
}
public Stat setData(String path, byte[] data, int version, long zxid, long time) throws KeeperException.NoNodeException {
@@ -635,8 +644,10 @@ public Stat setData(String path, byte[] data, int version, long zxid, long time)
if (n == null) {
throw new KeeperException.NoNodeException();
}
+ List acl;
byte[] lastdata = null;
synchronized (n) {
+ acl = getACL(n);
lastdata = n.data;
nodes.preChange(path, n);
n.data = data;
@@ -658,7 +669,7 @@ public Stat setData(String path, byte[] data, int version, long zxid, long time)
nodeDataSize.addAndGet(getNodeSize(path, data) - getNodeSize(path, lastdata));
updateWriteStat(path, dataBytes);
- dataWatches.triggerWatch(path, EventType.NodeDataChanged);
+ dataWatches.triggerWatch(path, EventType.NodeDataChanged, acl);
return s;
}
@@ -980,6 +991,7 @@ public ProcessTxnResult processTxn(TxnHeader header, Record txn, boolean isSubTx
Record record = null;
switch (subtxn.getType()) {
case OpCode.create:
+ case OpCode.create2:
record = new CreateTxn();
break;
case OpCode.createTTL:
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DumbWatcher.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DumbWatcher.java
index c7bf830b4ef..f78bd8ae516 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DumbWatcher.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DumbWatcher.java
@@ -22,8 +22,10 @@
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.security.cert.Certificate;
+import java.util.List;
import org.apache.jute.Record;
import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.ReplyHeader;
@@ -48,7 +50,7 @@ void setSessionTimeout(int sessionTimeout) {
}
@Override
- public void process(WatchedEvent event) {
+ public void process(WatchedEvent event, List znodeAcl) {
}
@Override
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
index e73f85d7cc7..e6c06d879e2 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
@@ -363,13 +363,22 @@ public void processRequest(Request request) {
}
case OpCode.exists: {
lastOp = "EXIS";
- // TODO we need to figure out the security requirement for this!
ExistsRequest existsRequest = new ExistsRequest();
ByteBufferInputStream.byteBuffer2Record(request.request, existsRequest);
path = existsRequest.getPath();
if (path.indexOf('\0') != -1) {
throw new KeeperException.BadArgumentsException();
}
+ DataNode n = zks.getZKDatabase().getNode(path);
+ if (n != null) {
+ zks.checkACL(
+ request.cnxn,
+ zks.getZKDatabase().aclForNode(n),
+ ZooDefs.Perms.READ,
+ request.authInfo,
+ path,
+ null);
+ }
Stat stat = zks.getZKDatabase().statNode(path, existsRequest.getWatch() ? cnxn : null);
rsp = new ExistsResponse(stat);
requestPathMetricsCollector.registerRequest(request.type, path);
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java
index 02cde23a917..9c31ca1fe62 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java
@@ -30,14 +30,17 @@
import java.nio.channels.SelectionKey;
import java.nio.channels.SocketChannel;
import java.security.cert.Certificate;
+import java.util.List;
import java.util.Queue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.jute.BinaryInputArchive;
import org.apache.jute.Record;
import org.apache.zookeeper.ClientCnxn;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.ReplyHeader;
@@ -330,7 +333,15 @@ void doIO(SelectionKey k) throws InterruptedException {
if (k.isReadable()) {
int rc = sock.read(incomingBuffer);
if (rc < 0) {
- handleFailedRead();
+ try {
+ handleFailedRead();
+ } catch (EndOfStreamException e) {
+ // no stacktrace. this case is very common, and it is usually not a problem.
+ LOG.info("{}", e.getMessage());
+ // expecting close to log session closure
+ close(e.getReason());
+ return;
+ }
}
if (incomingBuffer.remaining() == 0) {
boolean isPayload;
@@ -689,7 +700,18 @@ public int sendResponse(ReplyHeader h, Record r, String tag, String cacheKey, St
* @see org.apache.zookeeper.server.ServerCnxnIface#process(org.apache.zookeeper.proto.WatcherEvent)
*/
@Override
- public void process(WatchedEvent event) {
+ public void process(WatchedEvent event, List znodeAcl) {
+ try {
+ zkServer.checkACL(this, znodeAcl, ZooDefs.Perms.READ, getAuthInfo(), event.getPath(), null);
+ } catch (KeeperException.NoAuthException e) {
+ if (LOG.isTraceEnabled()) {
+ ZooTrace.logTraceMessage(
+ LOG,
+ ZooTrace.EVENT_DELIVERY_TRACE_MASK,
+ "Not delivering event " + event + " to 0x" + Long.toHexString(this.sessionId) + " (filtered by ACL)");
+ }
+ return;
+ }
ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, -1L, 0);
if (LOG.isTraceEnabled()) {
ZooTrace.logTraceMessage(
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java
index 8937039bca7..9ce11c8f837 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java
@@ -38,11 +38,15 @@
import java.nio.channels.SelectionKey;
import java.security.cert.Certificate;
import java.util.Arrays;
+import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.jute.BinaryInputArchive;
import org.apache.jute.Record;
import org.apache.zookeeper.ClientCnxn;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.ReplyHeader;
@@ -159,7 +163,18 @@ public int getSessionTimeout() {
}
@Override
- public void process(WatchedEvent event) {
+ public void process(WatchedEvent event, List znodeAcl) {
+ try {
+ zkServer.checkACL(this, znodeAcl, ZooDefs.Perms.READ, getAuthInfo(), event.getPath(), null);
+ } catch (KeeperException.NoAuthException e) {
+ if (LOG.isTraceEnabled()) {
+ ZooTrace.logTraceMessage(
+ LOG,
+ ZooTrace.EVENT_DELIVERY_TRACE_MASK,
+ "Not delivering event " + event + " to 0x" + Long.toHexString(this.sessionId) + " (filtered by ACL)");
+ }
+ return;
+ }
ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, -1L, 0);
if (LOG.isTraceEnabled()) {
ZooTrace.logTraceMessage(
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java
index 040650e4aaa..de580d7acc8 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java
@@ -565,7 +565,7 @@ private synchronized void initSSL(ChannelPipeline p, boolean supportPlaintext) t
SslContext nettySslContext;
if (authProviderProp == null) {
SSLContextAndOptions sslContextAndOptions = x509Util.getDefaultSSLContextAndOptions();
- nettySslContext = sslContextAndOptions.createNettyJdkSslContext(sslContextAndOptions.getSSLContext(), false);
+ nettySslContext = sslContextAndOptions.createNettyJdkSslContext(sslContextAndOptions.getSSLContext());
} else {
SSLContext sslContext = SSLContext.getInstance(ClientX509Util.DEFAULT_PROTOCOL);
X509AuthenticationProvider authProvider = (X509AuthenticationProvider) ProviderRegistry.getProvider(
@@ -577,7 +577,7 @@ private synchronized void initSSL(ChannelPipeline p, boolean supportPlaintext) t
}
sslContext.init(new X509KeyManager[]{authProvider.getKeyManager()}, new X509TrustManager[]{authProvider.getTrustManager()}, null);
- nettySslContext = x509Util.getDefaultSSLContextAndOptions().createNettyJdkSslContext(sslContext, false);
+ nettySslContext = x509Util.getDefaultSSLContextAndOptions().createNettyJdkSslContext(sslContext);
}
if (supportPlaintext) {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java
index d152fabd8dd..615aa3d958d 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java
@@ -135,7 +135,7 @@ public boolean accept(File f) {
}
// add all non-excluded log files
- File[] logs = txnLog.getDataDir().listFiles(new MyFileFilter(PREFIX_LOG));
+ File[] logs = txnLog.getDataLogDir().listFiles(new MyFileFilter(PREFIX_LOG));
List files = new ArrayList<>();
if (logs != null) {
files.addAll(Arrays.asList(logs));
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java
index d60efa0878f..4a401e5b919 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java
@@ -195,13 +195,11 @@ public void run() {
LOG.info("RequestThrottler shutdown. Dropped {} requests", dropped);
}
- private synchronized void throttleSleep(int stallTime) {
- try {
- ServerMetrics.getMetrics().REQUEST_THROTTLE_WAIT_COUNT.add(1);
- this.wait(stallTime);
- } catch (InterruptedException ie) {
- return;
- }
+
+ // @VisibleForTesting
+ synchronized void throttleSleep(int stallTime) throws InterruptedException {
+ ServerMetrics.getMetrics().REQUEST_THROTTLE_WAIT_COUNT.add(1);
+ this.wait(stallTime);
}
@SuppressFBWarnings(value = "NN_NAKED_NOTIFY", justification = "state change is in ZooKeeperServer.decInProgress() ")
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java
index b5b2645838d..7282c17150e 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java
@@ -39,8 +39,8 @@
import org.apache.jute.Record;
import org.apache.zookeeper.Quotas;
import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs.OpCode;
+import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.metrics.Counter;
@@ -53,7 +53,7 @@
* Interface to a Server connection - represents a connection from a client
* to the server.
*/
-public abstract class ServerCnxn implements Stats, Watcher {
+public abstract class ServerCnxn implements Stats, ServerWatcher {
// This is just an arbitrary object to represent requests issued by
// (aka owned by) this class
@@ -264,7 +264,11 @@ protected ByteBuffer[] serialize(ReplyHeader h, Record r, String tag,
/* notify the client the session is closing and close/cleanup socket */
public abstract void sendCloseSession();
- public abstract void process(WatchedEvent event);
+ public void process(WatchedEvent event) {
+ process(event, null);
+ }
+
+ public abstract void process(WatchedEvent event, List znodeAcl);
public abstract long getSessionId();
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerWatcher.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerWatcher.java
new file mode 100644
index 00000000000..bfd4b2501dc
--- /dev/null
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerWatcher.java
@@ -0,0 +1,29 @@
+/*
+ * 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.zookeeper.server;
+
+import java.util.List;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
+
+public interface ServerWatcher extends Watcher {
+
+ void process(WatchedEvent event, List znodeAcl);
+
+}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
index 86c13aea386..d017d295a64 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
@@ -110,7 +110,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
// When enabled, will check ACL constraints appertained to the requests first,
// before sending the requests to the quorum.
- static final boolean enableEagerACLCheck;
+ static boolean enableEagerACLCheck;
static final boolean skipACL;
@@ -157,6 +157,17 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
LOG.info("{} = {}", CLOSE_SESSION_TXN_ENABLED, closeSessionTxnEnabled);
}
+ // @VisibleForTesting
+ public static boolean isEnableEagerACLCheck() {
+ return enableEagerACLCheck;
+ }
+
+ // @VisibleForTesting
+ public static void setEnableEagerACLCheck(boolean enabled) {
+ ZooKeeperServer.enableEagerACLCheck = enabled;
+ LOG.info("Update {} to {}", ENABLE_EAGER_ACL_CHECK, enabled);
+ }
+
public static boolean isCloseSessionTxnEnabled() {
return closeSessionTxnEnabled;
}
@@ -374,7 +385,7 @@ public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, int minSessio
getMinSessionTimeout(),
getMaxSessionTimeout(),
getClientPortListenBacklog(),
- txnLogFactory.getDataDir(),
+ txnLogFactory.getDataLogDir(),
txnLogFactory.getSnapDir());
}
@@ -427,7 +438,7 @@ public void dumpConf(PrintWriter pwriter) {
pwriter.print("dataDirSize=");
pwriter.println(getDataDirSize());
pwriter.print("dataLogDir=");
- pwriter.println(zkDb.snapLog.getDataDir().getAbsolutePath());
+ pwriter.println(zkDb.snapLog.getDataLogDir().getAbsolutePath());
pwriter.print("dataLogSize=");
pwriter.println(getLogDirSize());
pwriter.print("tickTime=");
@@ -449,7 +460,7 @@ public ZooKeeperServerConf getConf() {
return new ZooKeeperServerConf(
getClientPort(),
zkDb.snapLog.getSnapDir().getAbsolutePath(),
- zkDb.snapLog.getDataDir().getAbsolutePath(),
+ zkDb.snapLog.getDataLogDir().getAbsolutePath(),
getTickTime(),
getMaxClientCnxnsPerHost(),
getMinSessionTimeout(),
@@ -557,7 +568,7 @@ public long getDataDirSize() {
if (zkDb == null) {
return 0L;
}
- File path = zkDb.snapLog.getDataDir();
+ File path = zkDb.snapLog.getSnapDir();
return getDirSize(path);
}
@@ -566,7 +577,7 @@ public long getLogDirSize() {
if (zkDb == null) {
return 0L;
}
- File path = zkDb.snapLog.getSnapDir();
+ File path = zkDb.snapLog.getDataLogDir();
return getDirSize(path);
}
@@ -738,9 +749,12 @@ protected void startJvmPauseMonitor() {
}
protected void startRequestThrottler() {
- requestThrottler = new RequestThrottler(this);
+ requestThrottler = createRequestThrottler();
requestThrottler.start();
+ }
+ protected RequestThrottler createRequestThrottler() {
+ return new RequestThrottler(this);
}
protected void setupRequestProcessors() {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
index 705c6cc3645..236c7ec2452 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
@@ -827,7 +827,7 @@ public CommandResponse run(ZooKeeperServer zkServer, Map kwargs)
QuorumPeer.ZabState zabState = peer.getZabState();
QuorumVerifier qv = peer.getQuorumVerifier();
- QuorumPeer.QuorumServer voter = qv.getVotingMembers().get(peer.getId());
+ QuorumPeer.QuorumServer voter = qv.getVotingMembers().get(peer.getMyId());
boolean voting = (
voter != null
&& voter.addr.equals(peer.getQuorumAddress())
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java
index 255e5cf8ae4..52eb7a7a988 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java
@@ -98,6 +98,7 @@ public X509AuthenticationProvider() throws X509Exception {
x509Util.getSslTruststorePasswdProperty(),
x509Util.getSslTruststorePasswdPathProperty());
String trustStoreTypeProp = config.getProperty(x509Util.getSslTruststoreTypeProperty());
+ boolean fipsMode = x509Util.getFipsMode(config);
if (trustStoreLocation.isEmpty()) {
LOG.warn("Truststore not specified for client connection");
@@ -110,7 +111,8 @@ public X509AuthenticationProvider() throws X509Exception {
crlEnabled,
ocspEnabled,
hostnameVerificationEnabled,
- false);
+ false,
+ fipsMode);
} catch (TrustManagerException e) {
LOG.error("Failed to create trust manager", e);
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
index 403720bdaa8..9cf8af1fd65 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
@@ -204,11 +204,11 @@ public boolean accept(File dir, String name) {
}
/**
- * get the datadir used by this filetxn
+ * get the data log dir used by this filetxn
* snap log
- * @return the data dir
+ * @return the data log dir
*/
- public File getDataDir() {
+ public File getDataLogDir() {
return this.dataDir;
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AckRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AckRequestProcessor.java
index 32eb8a8186b..f277dcdf3ff 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AckRequestProcessor.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AckRequestProcessor.java
@@ -44,7 +44,7 @@ public void processRequest(Request request) {
QuorumPeer self = leader.self;
if (self != null) {
request.logLatency(ServerMetrics.getMetrics().PROPOSAL_ACK_CREATION_LATENCY);
- leader.processAck(self.getId(), request.zxid, null);
+ leader.processAck(self.getMyId(), request.zxid, null);
} else {
LOG.error("Null QuorumPeer");
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java
index 86dce2b7f85..11ba0936122 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java
@@ -213,12 +213,11 @@ public void run() {
* request from a client on another server (i.e., the order of
* the following two lines is important!).
*/
- commitIsWaiting = !committedRequests.isEmpty();
- requestsToProcess = queuedRequests.size();
- // Avoid sync if we have something to do
- if (requestsToProcess == 0 && !commitIsWaiting) {
- // Waiting for requests to process
- synchronized (this) {
+ synchronized (this) {
+ commitIsWaiting = !committedRequests.isEmpty();
+ requestsToProcess = queuedRequests.size();
+ if (requestsToProcess == 0 && !commitIsWaiting) {
+ // Waiting for requests to process
while (!stopped && requestsToProcess == 0 && !commitIsWaiting) {
wait();
commitIsWaiting = !committedRequests.isEmpty();
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java
index 654ed1cef09..91207d110f4 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java
@@ -307,7 +307,7 @@ public void run() {
QuorumVerifier curQV = self.getQuorumVerifier();
if (rqv.getVersion() > curQV.getVersion()) {
LOG.info("{} Received version: {} my version: {}",
- self.getId(),
+ self.getMyId(),
Long.toHexString(rqv.getVersion()),
Long.toHexString(self.getQuorumVerifier().getVersion()));
if (self.getPeerState() == ServerState.LOOKING) {
@@ -356,7 +356,7 @@ public void run() {
sendqueue.offer(notmsg);
} else {
// Receive new message
- LOG.debug("Receive new notification message. My id = {}", self.getId());
+ LOG.debug("Receive new notification message. My id = {}", self.getMyId());
// State of peer that sent this message
QuorumPeer.ServerState ackstate = QuorumPeer.ServerState.LOOKING;
@@ -446,7 +446,7 @@ public void run() {
LOG.debug(
"Sending new notification. My id ={} recipient={} zxid=0x{} leader={} config version = {}",
- self.getId(),
+ self.getMyId(),
response.sid,
Long.toHexString(current.getZxid()),
current.getId(),
@@ -535,12 +535,12 @@ void process(ToSend m) {
this.ws = new WorkerSender(manager);
- this.wsThread = new Thread(this.ws, "WorkerSender[myid=" + self.getId() + "]");
+ this.wsThread = new Thread(this.ws, "WorkerSender[myid=" + self.getMyId() + "]");
this.wsThread.setDaemon(true);
this.wr = new WorkerReceiver(manager);
- this.wrThread = new Thread(this.wr, "WorkerReceiver[myid=" + self.getId() + "]");
+ this.wrThread = new Thread(this.wr, "WorkerReceiver[myid=" + self.getMyId() + "]");
this.wrThread.setDaemon(true);
}
@@ -663,7 +663,7 @@ private void leaveInstance(Vote v) {
"About to leave FLE instance: leader={}, zxid=0x{}, my id={}, my state={}",
v.getId(),
Long.toHexString(v.getZxid()),
- self.getId(),
+ self.getMyId(),
self.getPeerState());
recvqueue.clear();
}
@@ -708,7 +708,7 @@ private void sendNotifications() {
Long.toHexString(proposedZxid),
Long.toHexString(logicalclock.get()),
sid,
- self.getId(),
+ self.getMyId(),
Long.toHexString(proposedEpoch));
sendqueue.offer(notmsg);
@@ -800,7 +800,7 @@ protected boolean checkLeader(Map votes, long leader, long electionE
* from leader stating that it is leading, then predicate is false.
*/
- if (leader != self.getId()) {
+ if (leader != self.getMyId()) {
if (votes.get(leader) == null) {
predicate = false;
} else if (votes.get(leader).getState() != ServerState.LEADING) {
@@ -839,10 +839,10 @@ public synchronized Vote getVote() {
*/
private ServerState learningState() {
if (self.getLearnerType() == LearnerType.PARTICIPANT) {
- LOG.debug("I am a participant: {}", self.getId());
+ LOG.debug("I am a participant: {}", self.getMyId());
return ServerState.FOLLOWING;
} else {
- LOG.debug("I am an observer: {}", self.getId());
+ LOG.debug("I am an observer: {}", self.getMyId());
return ServerState.OBSERVING;
}
}
@@ -853,8 +853,8 @@ private ServerState learningState() {
* @return long
*/
private long getInitId() {
- if (self.getQuorumVerifier().getVotingMembers().containsKey(self.getId())) {
- return self.getId();
+ if (self.getQuorumVerifier().getVotingMembers().containsKey(self.getMyId())) {
+ return self.getMyId();
} else {
return Long.MIN_VALUE;
}
@@ -897,7 +897,7 @@ private long getPeerEpoch() {
* the leadingVoteSet if it becomes the leader.
*/
private void setPeerState(long proposedLeader, SyncedLearnerTracker voteSet) {
- ServerState ss = (proposedLeader == self.getId()) ? ServerState.LEADING : learningState();
+ ServerState ss = (proposedLeader == self.getMyId()) ? ServerState.LEADING : learningState();
self.setPeerState(ss);
if (ss == ServerState.LEADING) {
leadingVoteSet = voteSet;
@@ -945,7 +945,7 @@ public Vote lookForLeader() throws InterruptedException {
LOG.info(
"New election. My id = {}, proposed zxid=0x{}",
- self.getId(),
+ self.getMyId(),
Long.toHexString(proposedZxid));
sendNotifications();
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
index ce8f7999c45..8cb981237cf 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
@@ -271,8 +271,8 @@ boolean isLearnerSynced(LearnerHandler peer) {
*/
public boolean isQuorumSynced(QuorumVerifier qv) {
HashSet ids = new HashSet();
- if (qv.getVotingMembers().containsKey(self.getId())) {
- ids.add(self.getId());
+ if (qv.getVotingMembers().containsKey(self.getMyId())) {
+ ids.add(self.getMyId());
}
synchronized (forwardingFollowers) {
for (LearnerHandler learnerHandler : forwardingFollowers) {
@@ -310,6 +310,10 @@ public Leader(QuorumPeer self, LeaderZooKeeperServer zk) throws IOException {
this.zk = zk;
}
+ InetSocketAddress recreateInetSocketAddr(String hostString, int port) {
+ return new InetSocketAddress(hostString, port);
+ }
+
Optional createServerSocket(InetSocketAddress address, boolean portUnification, boolean sslQuorum) {
ServerSocket serverSocket;
try {
@@ -319,7 +323,7 @@ Optional createServerSocket(InetSocketAddress address, boolean por
serverSocket = new ServerSocket();
}
serverSocket.setReuseAddress(true);
- serverSocket.bind(address);
+ serverSocket.bind(recreateInetSocketAddr(address.getHostString(), address.getPort()));
return Optional.of(serverSocket);
} catch (IOException e) {
LOG.error("Couldn't bind to {}", address.toString(), e);
@@ -603,7 +607,7 @@ void lead() throws IOException, InterruptedException {
cnxAcceptor = new LearnerCnxAcceptor();
cnxAcceptor.start();
- long epoch = getEpochToPropose(self.getId(), self.getAcceptedEpoch());
+ long epoch = getEpochToPropose(self.getMyId(), self.getAcceptedEpoch());
zk.setZxid(ZxidUtils.makeZxid(epoch, 0));
@@ -658,13 +662,13 @@ void lead() throws IOException, InterruptedException {
// us. We do this by waiting for the NEWLEADER packet to get
// acknowledged
- waitForEpochAck(self.getId(), leaderStateSummary);
+ waitForEpochAck(self.getMyId(), leaderStateSummary);
self.setCurrentEpoch(epoch);
- self.setLeaderAddressAndId(self.getQuorumAddress(), self.getId());
+ self.setLeaderAddressAndId(self.getQuorumAddress(), self.getMyId());
self.setZabState(QuorumPeer.ZabState.SYNCHRONIZATION);
try {
- waitForNewLeaderAck(self.getId(), zk.getZxid());
+ waitForNewLeaderAck(self.getMyId(), zk.getZxid());
} catch (InterruptedException e) {
shutdown("Waiting for a quorum of followers, only synced with sids: [ "
+ newLeaderProposal.ackSetsToString()
@@ -746,7 +750,7 @@ void lead() throws IOException, InterruptedException {
syncedAckSet.addQuorumVerifier(self.getLastSeenQuorumVerifier());
}
- syncedAckSet.addAck(self.getId());
+ syncedAckSet.addAck(self.getMyId());
for (LearnerHandler f : getLearners()) {
if (f.synced()) {
@@ -870,15 +874,15 @@ private long getDesignatedLeader(Proposal reconfigProposal, long zxid) {
//check if I'm in the new configuration with the same quorum address -
// if so, I'll remain the leader
- if (newQVAcksetPair.getQuorumVerifier().getVotingMembers().containsKey(self.getId())
- && newQVAcksetPair.getQuorumVerifier().getVotingMembers().get(self.getId()).addr.equals(self.getQuorumAddress())) {
- return self.getId();
+ if (newQVAcksetPair.getQuorumVerifier().getVotingMembers().containsKey(self.getMyId())
+ && newQVAcksetPair.getQuorumVerifier().getVotingMembers().get(self.getMyId()).addr.equals(self.getQuorumAddress())) {
+ return self.getMyId();
}
// start with an initial set of candidates that are voters from new config that
// acknowledged the reconfig op (there must be a quorum). Choose one of them as
// current leader candidate
HashSet candidates = new HashSet(newQVAcksetPair.getAckset());
- candidates.remove(self.getId()); // if we're here, I shouldn't be the leader
+ candidates.remove(self.getMyId()); // if we're here, I shouldn't be the leader
long curCandidate = candidates.iterator().next();
//go over outstanding ops in order, and try to find a candidate that acked the most ops.
@@ -960,7 +964,7 @@ public synchronized boolean tryToCommit(Proposal p, long zxid, SocketAddress fol
self.processReconfig(newQV, designatedLeader, zk.getZxid(), true);
- if (designatedLeader != self.getId()) {
+ if (designatedLeader != self.getMyId()) {
LOG.info(String.format("Committing a reconfiguration (reconfigEnabled=%s); this leader is not the designated "
+ "leader anymore, setting allowedToCommit=false", self.isReconfigEnabled()));
allowedToCommit = false;
@@ -1424,13 +1428,13 @@ public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws Interrupt
connectingFollowers.add(sid);
}
QuorumVerifier verifier = self.getQuorumVerifier();
- if (connectingFollowers.contains(self.getId()) && verifier.containsQuorum(connectingFollowers)) {
+ if (connectingFollowers.contains(self.getMyId()) && verifier.containsQuorum(connectingFollowers)) {
waitingForNewEpoch = false;
self.setAcceptedEpoch(epoch);
connectingFollowers.notifyAll();
} else {
long start = Time.currentElapsedTime();
- if (sid == self.getId()) {
+ if (sid == self.getMyId()) {
timeStartWaitForEpoch = start;
}
long cur = start;
@@ -1476,7 +1480,7 @@ public void waitForEpochAck(long id, StateSummary ss) throws IOException, Interr
}
}
QuorumVerifier verifier = self.getQuorumVerifier();
- if (electingFollowers.contains(self.getId()) && verifier.containsQuorum(electingFollowers)) {
+ if (electingFollowers.contains(self.getMyId()) && verifier.containsQuorum(electingFollowers)) {
electionFinished = true;
electingFollowers.notifyAll();
} else {
@@ -1533,7 +1537,7 @@ private synchronized void startZkServer() {
Long designatedLeader = getDesignatedLeader(newLeaderProposal, zk.getZxid());
self.processReconfig(newQV, designatedLeader, zk.getZxid(), true);
- if (designatedLeader != self.getId()) {
+ if (designatedLeader != self.getMyId()) {
LOG.warn("This leader is not the designated leader, it will be initialized with allowedToCommit = false");
allowedToCommit = false;
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java
index 5a51bc23d7e..2de080bd380 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java
@@ -175,7 +175,7 @@ public void createSessionTracker() {
this,
getZKDatabase().getSessionWithTimeOuts(),
tickTime,
- self.getId(),
+ self.getMyId(),
self.areLocalSessionsEnabled(),
getZooKeeperServerListener());
}
@@ -291,7 +291,7 @@ public String getState() {
*/
@Override
public long getServerId() {
- return self.getId();
+ return self.getMyId();
}
@Override
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
index 594c87fb90d..44cbdf2a2a6 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
@@ -495,7 +495,7 @@ protected long registerWithLeader(int pktType) throws IOException {
/*
* Add sid to payload
*/
- LearnerInfo li = new LearnerInfo(self.getId(), 0x10000, self.getQuorumVerifier().getVersion());
+ LearnerInfo li = new LearnerInfo(self.getMyId(), 0x10000, self.getQuorumVerifier().getVersion());
ByteArrayOutputStream bsid = new ByteArrayOutputStream();
BinaryOutputArchive boa = BinaryOutputArchive.getArchive(bsid);
boa.writeRecord(li, "LearnerInfo");
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java
index cab95b05621..efd2e376208 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java
@@ -71,7 +71,7 @@ protected Map getTouchSnapshot() {
*/
@Override
public long getServerId() {
- return self.getId();
+ return self.getMyId();
}
@Override
@@ -80,7 +80,7 @@ public void createSessionTracker() {
this,
getZKDatabase().getSessionWithTimeOuts(),
this.tickTime,
- self.getId(),
+ self.getMyId(),
self.areLocalSessionsEnabled(),
getZooKeeperServerListener());
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java
index f5d0dbedaf9..f1649ff93e2 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java
@@ -35,7 +35,7 @@ public LocalPeerBean(QuorumPeer peer) {
}
public String getName() {
- return "replica." + peer.getId();
+ return "replica." + peer.getMyId();
}
public boolean isHidden() {
@@ -119,12 +119,12 @@ public String getQuorumSystemInfo() {
@Override
public boolean isPartOfEnsemble() {
- return peer.getView().containsKey(peer.getId());
+ return peer.getView().containsKey(peer.getMyId());
}
@Override
public boolean isLeader() {
- return peer.isLeader(peer.getId());
+ return peer.isLeader(peer.getMyId());
}
@Override
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java
index aae4552f498..46db6865b8a 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java
@@ -28,7 +28,7 @@ public class QuorumBean implements QuorumMXBean, ZKMBeanInfo {
public QuorumBean(QuorumPeer peer) {
this.peer = peer;
- name = "ReplicatedServer_id" + peer.getId();
+ name = "ReplicatedServer_id" + peer.getMyId();
}
@Override
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
index 9d9f8bd86a0..eb5346af4bf 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
@@ -466,7 +466,7 @@ public void run() {
private boolean startConnection(Socket sock, Long sid) throws IOException {
DataOutputStream dout = null;
DataInputStream din = null;
- LOG.debug("startConnection (myId:{} --> sid:{})", self.getId(), sid);
+ LOG.debug("startConnection (myId:{} --> sid:{})", self.getMyId(), sid);
try {
// Use BufferedOutputStream to reduce the number of IP packets. This is
// important for x-DC scenarios.
@@ -481,7 +481,7 @@ private boolean startConnection(Socket sock, Long sid) throws IOException {
// understand the protocol version we use to avoid multiple partitions. see ZOOKEEPER-3720
long protocolVersion = self.isMultiAddressEnabled() ? PROTOCOL_VERSION_V2 : PROTOCOL_VERSION_V1;
dout.writeLong(protocolVersion);
- dout.writeLong(self.getId());
+ dout.writeLong(self.getMyId());
// now we send our election address. For the new protocol version, we can send multiple addresses.
Collection addressesToSend = protocolVersion == PROTOCOL_VERSION_V2
@@ -510,12 +510,12 @@ private boolean startConnection(Socket sock, Long sid) throws IOException {
}
// If lost the challenge, then drop the new connection
- if (sid > self.getId()) {
- LOG.info("Have smaller server identifier, so dropping the connection: (myId:{} --> sid:{})", self.getId(), sid);
+ if (sid > self.getMyId()) {
+ LOG.info("Have smaller server identifier, so dropping the connection: (myId:{} --> sid:{})", self.getMyId(), sid);
closeSocket(sock);
// Otherwise proceed with the connection
} else {
- LOG.debug("Have larger server identifier, so keeping the connection: (myId:{} --> sid:{})", self.getId(), sid);
+ LOG.debug("Have larger server identifier, so keeping the connection: (myId:{} --> sid:{})", self.getMyId(), sid);
SendWorker sw = new SendWorker(sock, sid);
RecvWorker rw = new RecvWorker(sock, din, sid, sw);
sw.setRecv(rw);
@@ -610,7 +610,7 @@ private void handleConnection(Socket sock, DataInputStream din) throws IOExcepti
electionAddr = new MultipleAddresses(init.electionAddr,
Duration.ofMillis(self.getMultiAddressReachabilityCheckTimeoutMs()));
}
- LOG.debug("Initial message parsed by {}: {}", self.getId(), init.toString());
+ LOG.debug("Initial message parsed by {}: {}", self.getMyId(), init.toString());
} catch (InitialMessage.InitialMessageException ex) {
LOG.error("Initial message parsing error!", ex);
closeSocket(sock);
@@ -635,7 +635,7 @@ private void handleConnection(Socket sock, DataInputStream din) throws IOExcepti
// do authenticating learner
authServer.authenticate(sock, din);
//If wins the challenge, then close the new connection.
- if (sid < self.getId()) {
+ if (sid < self.getMyId()) {
/*
* This replica might still believe that the connection to sid is
* up, so we have to shut down the workers before trying to open a
@@ -658,7 +658,7 @@ private void handleConnection(Socket sock, DataInputStream din) throws IOExcepti
connectOne(sid);
}
- } else if (sid == self.getId()) {
+ } else if (sid == self.getMyId()) {
// we saw this case in ZOOKEEPER-2164
LOG.warn("We got a connection request from a server with our own ID. "
+ "This should be either a configuration error, or a bug.");
@@ -760,7 +760,7 @@ synchronized void connectOne(long sid) {
Map lastProposedView = lastSeenQV.getAllMembers();
if (lastCommittedView.containsKey(sid)) {
knownId = true;
- LOG.debug("Server {} knows {} already, it is in the lastCommittedView", self.getId(), sid);
+ LOG.debug("Server {} knows {} already, it is in the lastCommittedView", self.getMyId(), sid);
if (connectOne(sid, lastCommittedView.get(sid).electionAddr)) {
return;
}
@@ -770,7 +770,7 @@ synchronized void connectOne(long sid) {
&& (!knownId
|| !lastProposedView.get(sid).electionAddr.equals(lastCommittedView.get(sid).electionAddr))) {
knownId = true;
- LOG.debug("Server {} knows {} already, it is in the lastProposedView", self.getId(), sid);
+ LOG.debug("Server {} knows {} already, it is in the lastProposedView", self.getMyId(), sid);
if (connectOne(sid, lastProposedView.get(sid).electionAddr)) {
return;
@@ -839,7 +839,7 @@ public void halt() {
*/
public void softHalt() {
for (SendWorker sw : senderWorkerMap.values()) {
- LOG.debug("Server {} is soft-halting sender towards: {}", self.getId(), sw);
+ LOG.debug("Server {} is soft-halting sender towards: {}", self.getMyId(), sw);
sw.finish();
}
}
@@ -943,7 +943,7 @@ void setSocketBindErrorHandler(Runnable errorHandler) {
@Override
public void run() {
if (!shutdown) {
- LOG.debug("Listener thread started, myId: {}", self.getId());
+ LOG.debug("Listener thread started, myId: {}", self.getMyId());
Set addresses;
if (self.getQuorumListenOnAllIPs()) {
@@ -1297,7 +1297,7 @@ public void run() {
}
this.finish();
- LOG.warn("Send worker leaving thread id {} my id = {}", sid, self.getId());
+ LOG.warn("Send worker leaving thread id {} my id = {}", sid, self.getMyId());
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
index 18e97bb2f0c..404aaffcbde 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
@@ -628,7 +628,7 @@ public void setJvmPauseMonitor(JvmPauseMonitor jvmPauseMonitor) {
/**
* get the id of this quorum peer.
*/
- public long getId() {
+ public long getMyId() {
return myid;
}
@@ -990,7 +990,7 @@ public void recreateSocketAddresses(long id) {
QuorumServer qs = qv.getAllMembers().get(id);
if (qs != null) {
qs.recreateSocketAddresses();
- if (id == getId()) {
+ if (id == getMyId()) {
setAddrs(qs.addr, qs.electionAddr, qs.clientAddr);
}
}
@@ -1418,7 +1418,7 @@ public void run() {
MBeanRegistry.getInstance().register(jmxQuorumBean, null);
for (QuorumServer s : getView().values()) {
ZKMBeanInfo p;
- if (getId() == s.id) {
+ if (getMyId() == s.id) {
p = jmxLocalPeerBean = new LocalPeerBean(this);
try {
MBeanRegistry.getInstance().register(p, jmxQuorumBean);
@@ -1586,7 +1586,7 @@ private synchronized void updateServerState() {
return;
}
- if (getId() == getCurrentVote().getId()) {
+ if (getMyId() == getCurrentVote().getId()) {
setPeerState(ServerState.LEADING);
LOG.debug("PeerState set to LEADING");
} else if (getLearnerType() == LearnerType.PARTICIPANT) {
@@ -1880,7 +1880,7 @@ private void connectNewPeers(QuorumCnxManager qcm) {
if (quorumVerifier != null && lastSeenQuorumVerifier != null) {
Map committedView = quorumVerifier.getAllMembers();
for (Entry e : lastSeenQuorumVerifier.getAllMembers().entrySet()) {
- if (e.getKey() != getId() && !committedView.containsKey(e.getKey())) {
+ if (e.getKey() != getMyId() && !committedView.containsKey(e.getKey())) {
qcm.connectOne(e.getKey());
}
}
@@ -1941,7 +1941,7 @@ public QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean writeToDisk)
// and then got the same config in UPTODATE message so its already known
LOG.debug(
"{} setQuorumVerifier called with known or old config {}. Current version: {}",
- getId(),
+ getMyId(),
qv.getVersion(),
quorumVerifier.getVersion());
return quorumVerifier;
@@ -1970,7 +1970,7 @@ public QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean writeToDisk)
if (qv.getVersion() == lastSeenQuorumVerifier.getVersion()) {
QuorumPeerConfig.deleteFile(getNextDynamicConfigFilename());
}
- QuorumServer qs = qv.getAllMembers().get(getId());
+ QuorumServer qs = qv.getAllMembers().get(getMyId());
if (qs != null) {
setAddrs(qs.addr, qs.electionAddr, qs.clientAddr);
}
@@ -1984,7 +1984,7 @@ private String makeDynamicConfigFilename(long version) {
}
private boolean needEraseClientInfoFromStaticConfig() {
- QuorumServer server = quorumVerifier.getAllMembers().get(getId());
+ QuorumServer server = quorumVerifier.getAllMembers().get(getMyId());
return (server != null && server.clientAddr != null && !server.isClientAddrFromStatic);
}
@@ -2268,7 +2268,7 @@ public boolean processReconfig(QuorumVerifier qv, Long suggestedLeaderId, Long z
restartLeaderElection(prevQV, qv);
}
- QuorumServer myNewQS = newMembers.get(getId());
+ QuorumServer myNewQS = newMembers.get(getMyId());
if (myNewQS != null && myNewQS.clientAddr != null && !myNewQS.clientAddr.equals(oldClientAddr)) {
cnxnFactory.reconfigure(myNewQS.clientAddr);
updateThreadName();
@@ -2308,7 +2308,7 @@ private void updateRemotePeerMXBeans(Map newMembers) {
Set joiningMembers = new HashSet(newMembers.keySet());
joiningMembers.removeAll(jmxRemotePeerBean.keySet());
- joiningMembers.remove(getId()); // remove self as it is local bean
+ joiningMembers.remove(getMyId()); // remove self as it is local bean
for (Long id : joiningMembers) {
QuorumServer qs = newMembers.get(id);
RemotePeerBean rBean = new RemotePeerBean(this, qs);
@@ -2421,7 +2421,7 @@ QuorumServer validateLearnerMaster(String desiredMaster) {
private boolean updateLearnerType(QuorumVerifier newQV) {
//check if I'm an observer in new config
- if (newQV.getObservingMembers().containsKey(getId())) {
+ if (newQV.getObservingMembers().containsKey(getMyId())) {
if (getLearnerType() != LearnerType.OBSERVER) {
setLearnerType(LearnerType.OBSERVER);
LOG.info("Becoming an observer");
@@ -2430,7 +2430,7 @@ private boolean updateLearnerType(QuorumVerifier newQV) {
} else {
return false;
}
- } else if (newQV.getVotingMembers().containsKey(getId())) {
+ } else if (newQV.getVotingMembers().containsKey(getMyId())) {
if (getLearnerType() != LearnerType.PARTICIPANT) {
setLearnerType(LearnerType.PARTICIPANT);
LOG.info("Becoming a voting participant");
@@ -2514,7 +2514,7 @@ private void updateThreadName() {
: "disabled"
: "disabled";
String secure = secureCnxnFactory != null ? formatInetAddr(secureCnxnFactory.getLocalAddress()) : "disabled";
- setName(String.format("QuorumPeer[myid=%d](plain=%s)(secure=%s)", getId(), plain, secure));
+ setName(String.format("QuorumPeer[myid=%d](plain=%s)(secure=%s)", getMyId(), plain, secure));
}
/**
@@ -2591,7 +2591,7 @@ public QuorumCnxManager createCnxnManager() {
LOG.info("Using {}ms as the quorum cnxn socket timeout", timeout);
return new QuorumCnxManager(
this,
- this.getId(),
+ this.getMyId(),
this.getView(),
this.authServer,
this.authLearner,
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java
index d2f6b39b69a..0efa4677053 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java
@@ -88,7 +88,7 @@ public synchronized void startup() {
public void createSessionTracker() {
sessionTracker = new LearnerSessionTracker(
this, getZKDatabase().getSessionWithTimeOuts(),
- this.tickTime, self.getId(), self.areLocalSessionsEnabled(),
+ this.tickTime, self.getMyId(), self.areLocalSessionsEnabled(),
getZooKeeperServerListener());
}
@@ -186,7 +186,7 @@ public String getState() {
*/
@Override
public long getServerId() {
- return self.getId();
+ return self.getMyId();
}
@Override
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumServerCallbackHandler.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumServerCallbackHandler.java
index 4b711a6b5f7..b182ee13a1c 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumServerCallbackHandler.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumServerCallbackHandler.java
@@ -19,6 +19,7 @@
package org.apache.zookeeper.server.quorum.auth;
import java.io.IOException;
+import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
@@ -31,6 +32,7 @@
import javax.security.auth.login.Configuration;
import javax.security.sasl.AuthorizeCallback;
import javax.security.sasl.RealmCallback;
+import org.apache.zookeeper.server.auth.DigestLoginModule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -46,7 +48,8 @@ public class SaslQuorumServerCallbackHandler implements CallbackHandler {
private static final Logger LOG = LoggerFactory.getLogger(SaslQuorumServerCallbackHandler.class);
private String userName;
- private final Map credentials = new HashMap<>();
+ private final boolean isDigestAuthn;
+ private final Map credentials;
private final Set authzHosts;
public SaslQuorumServerCallbackHandler(
@@ -60,20 +63,35 @@ public SaslQuorumServerCallbackHandler(
LOG.error(errorMessage);
throw new IOException(errorMessage);
}
- credentials.clear();
+
+ Map credentials = new HashMap<>();
+ boolean isDigestAuthn = true;
+
for (AppConfigurationEntry entry : configurationEntries) {
- Map options = entry.getOptions();
- // Populate DIGEST-MD5 user -> password map with JAAS configuration entries from the "QuorumServer" section.
- // Usernames are distinguished from other options by prefixing the username with a "user_" prefix.
- for (Map.Entry pair : options.entrySet()) {
- String key = pair.getKey();
- if (key.startsWith(USER_PREFIX)) {
- String userName = key.substring(USER_PREFIX.length());
- credentials.put(userName, (String) pair.getValue());
+ if (entry.getLoginModuleName().equals(DigestLoginModule.class.getName())) {
+ Map options = entry.getOptions();
+ // Populate DIGEST-MD5 user -> password map with JAAS configuration entries from the "QuorumServer" section.
+ // Usernames are distinguished from other options by prefixing the username with a "user_" prefix.
+ for (Map.Entry pair : options.entrySet()) {
+ String key = pair.getKey();
+ if (key.startsWith(USER_PREFIX)) {
+ String userName = key.substring(USER_PREFIX.length());
+ credentials.put(userName, (String) pair.getValue());
+ }
}
+ } else {
+ isDigestAuthn = false;
}
}
+ this.isDigestAuthn = isDigestAuthn;
+ if (isDigestAuthn) {
+ this.credentials = Collections.unmodifiableMap(credentials);
+ LOG.warn("Using DIGEST-MD5 for quorum authorization");
+ } else {
+ this.credentials = Collections.emptyMap();
+ }
+
// authorized host lists
this.authzHosts = authzHosts;
}
@@ -126,13 +144,15 @@ private void handleAuthorizeCallback(AuthorizeCallback ac) {
// 2. Verify whether the connecting host is present in authorized hosts.
// If not exists, then connecting peer is not authorized to join the
// ensemble and will reject it.
- if (authzFlag) {
+ if (!isDigestAuthn && authzFlag) {
String[] components = authorizationID.split("[/@]");
if (components.length == 3) {
authzFlag = authzHosts.contains(components[1]);
+ } else {
+ authzFlag = false;
}
if (!authzFlag) {
- LOG.error("SASL authorization completed, {} is not authorized to connect", components[1]);
+ LOG.error("SASL authorization completed, {} is not authorized to connect", authorizationID);
}
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java
index 1bc44c805a0..b612dd71f7c 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java
@@ -19,8 +19,10 @@
package org.apache.zookeeper.server.watch;
import java.io.PrintWriter;
+import java.util.List;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.data.ACL;
public interface IWatchManager {
@@ -82,10 +84,11 @@ default boolean addWatch(String path, Watcher watcher, WatcherMode watcherMode)
*
* @param path znode path
* @param type the watch event type
+ * @param acl ACL of the znode in path
*
* @return the watchers have been notified
*/
- WatcherOrBitSet triggerWatch(String path, EventType type);
+ WatcherOrBitSet triggerWatch(String path, EventType type, List acl);
/**
* Distribute the watch event for the given path, but ignore those
@@ -97,7 +100,7 @@ default boolean addWatch(String path, Watcher watcher, WatcherMode watcherMode)
*
* @return the watchers have been notified
*/
- WatcherOrBitSet triggerWatch(String path, EventType type, WatcherOrBitSet suppress);
+ WatcherOrBitSet triggerWatch(String path, EventType type, List acl, WatcherOrBitSet suppress);
/**
* Get the size of watchers.
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java
index c5b133059b2..0c24c73956e 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java
@@ -22,6 +22,7 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
@@ -29,8 +30,10 @@
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.EventType;
import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.server.ServerCnxn;
import org.apache.zookeeper.server.ServerMetrics;
+import org.apache.zookeeper.server.ServerWatcher;
import org.apache.zookeeper.server.ZooTrace;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -115,12 +118,12 @@ public synchronized void removeWatcher(Watcher watcher) {
}
@Override
- public WatcherOrBitSet triggerWatch(String path, EventType type) {
- return triggerWatch(path, type, null);
+ public WatcherOrBitSet triggerWatch(String path, EventType type, List acl) {
+ return triggerWatch(path, type, acl, null);
}
@Override
- public WatcherOrBitSet triggerWatch(String path, EventType type, WatcherOrBitSet supress) {
+ public WatcherOrBitSet triggerWatch(String path, EventType type, List acl, WatcherOrBitSet supress) {
WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path);
Set watchers = new HashSet<>();
PathParentIterator pathParentIterator = getPathParentIterator(path);
@@ -165,7 +168,11 @@ public WatcherOrBitSet triggerWatch(String path, EventType type, WatcherOrBitSet
if (supress != null && supress.contains(w)) {
continue;
}
- w.process(e);
+ if (w instanceof ServerWatcher) {
+ ((ServerWatcher) w).process(e, acl);
+ } else {
+ w.process(e);
+ }
}
switch (type) {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManagerOptimized.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManagerOptimized.java
index 1cc7deb9dc0..947a5b6da31 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManagerOptimized.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManagerOptimized.java
@@ -22,6 +22,7 @@
import java.util.BitSet;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
@@ -31,8 +32,10 @@
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.EventType;
import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.server.ServerCnxn;
import org.apache.zookeeper.server.ServerMetrics;
+import org.apache.zookeeper.server.ServerWatcher;
import org.apache.zookeeper.server.util.BitHashSet;
import org.apache.zookeeper.server.util.BitMap;
import org.slf4j.Logger;
@@ -202,12 +205,12 @@ public void processDeadWatchers(Set deadWatchers) {
}
@Override
- public WatcherOrBitSet triggerWatch(String path, EventType type) {
- return triggerWatch(path, type, null);
+ public WatcherOrBitSet triggerWatch(String path, EventType type, List acl) {
+ return triggerWatch(path, type, acl, null);
}
@Override
- public WatcherOrBitSet triggerWatch(String path, EventType type, WatcherOrBitSet suppress) {
+ public WatcherOrBitSet triggerWatch(String path, EventType type, List acl, WatcherOrBitSet suppress) {
WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path);
BitHashSet watchers = remove(path);
@@ -232,7 +235,11 @@ public WatcherOrBitSet triggerWatch(String path, EventType type, WatcherOrBitSet
continue;
}
- w.process(e);
+ if (w instanceof ServerWatcher) {
+ ((ServerWatcher) w).process(e, acl);
+ } else {
+ w.process(e);
+ }
triggeredWatches++;
}
}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/util/ServiceUtils.java b/zookeeper-server/src/main/java/org/apache/zookeeper/util/ServiceUtils.java
index 544e13d7f47..944bbc03cd9 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/util/ServiceUtils.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/util/ServiceUtils.java
@@ -39,7 +39,14 @@ private ServiceUtils() {
*/
@SuppressFBWarnings("DM_EXIT")
public static final Consumer SYSTEM_EXIT = (code) -> {
- LOG.error("Exiting JVM with code {}", code);
+ String msg = "Exiting JVM with code {}";
+ if (code == 0) {
+ // JVM exits normally
+ LOG.info(msg, code);
+ } else {
+ // JVM exits with error
+ LOG.error(msg, code);
+ }
System.exit(code);
};
@@ -47,8 +54,12 @@ private ServiceUtils() {
* No-op strategy, useful for tests.
*/
public static final Consumer LOG_ONLY = (code) -> {
- LOG.error("Fatal error, JVM should exit with code {}. "
+ if (code != 0) {
+ LOG.error("Fatal error, JVM should exit with code {}. "
+ "Actually System.exit is disabled", code);
+ } else {
+ LOG.info("JVM should exit with code {}. Actually System.exit is disabled", code);
+ }
};
private static volatile Consumer systemExitProcedure = SYSTEM_EXIT;
diff --git a/zookeeper-server/src/main/resources/NOTICE.txt b/zookeeper-server/src/main/resources/NOTICE.txt
index efdd6b43970..cff9c31d192 100644
--- a/zookeeper-server/src/main/resources/NOTICE.txt
+++ b/zookeeper-server/src/main/resources/NOTICE.txt
@@ -1,5 +1,5 @@
Apache ZooKeeper
-Copyright 2009-2021 The Apache Software Foundation
+Copyright 2009-2024 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
@@ -11,10 +11,10 @@ for Airlift code can be found at:
https://github.com/airlift/airlift/blob/master/LICENSE
This product includes software developed by
-The Netty Project (http://netty.io/)
-Copyright 2011 The Netty Project
-
-The Netty NOTICE file contains the following items:
+The Netty Project (http://netty.io/) Copyright 2011 The Netty Project
+The Netty NOTICE file (https://github.com/netty/netty/blob/4.1/NOTICE.txt)
+contains the following items:
+---------------- start of netty NOTICE file ----------------
This product contains the extensions to Java Collections Framework which has
been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
@@ -32,29 +32,112 @@ Base64 Encoder and Decoder, which can be obtained at:
* HOMEPAGE:
* http://iharder.sourceforge.net/current/java/base64/
-This product contains a modified version of 'JZlib', a re-implementation of
-zlib in pure Java, which can be obtained at:
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.webbit.txt (BSD License)
+ * HOMEPAGE:
+ * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.slf4j.txt (MIT License)
+ * HOMEPAGE:
+ * https://www.slf4j.org/
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+ * NOTICE:
+ * license/NOTICE.harmony.txt
* LICENSE:
- * license/LICENSE.jzlib.txt (BSD Style License)
+ * license/LICENSE.harmony.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://archive.apache.org/dist/harmony/
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jbzip2.txt (MIT License)
+ * HOMEPAGE:
+ * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.libdivsufsort.txt (MIT License)
+ * HOMEPAGE:
+ * https://github.com/y-256/libdivsufsort
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jctools.txt (ASL2 License)
+ * HOMEPAGE:
+ * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jzlib.txt (BSD style License)
* HOMEPAGE:
* http://www.jcraft.com/jzlib/
-This product contains a modified version of 'Webbit', a Java event based
-WebSocket and HTTP server:
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
* LICENSE:
- * license/LICENSE.webbit.txt (BSD License)
+ * license/LICENSE.compress-lzf.txt (Apache License 2.0)
* HOMEPAGE:
- * https://github.com/joewalnes/webbit
+ * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.lz4.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
-This product optionally depends on 'Protocol Buffers', Google's data
+ * LICENSE:
+ * license/LICENSE.lzma-java.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/jponge/lzma-java
+
+This product optionally depends on 'zstd-jni', a zstd-jni Java compression
+and decompression library, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.zstd-jni.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/luben/zstd-jni
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jfastlz.txt (MIT License)
+ * HOMEPAGE:
+ * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
interchange format, which can be obtained at:
* LICENSE:
* license/LICENSE.protobuf.txt (New BSD License)
* HOMEPAGE:
- * http://code.google.com/p/protobuf/
+ * https://github.com/google/protobuf
This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
a temporary self-signed X.509 certificate when the JVM does not provide the
@@ -63,15 +146,31 @@ equivalent functionality. It can be obtained at:
* LICENSE:
* license/LICENSE.bouncycastle.txt (MIT License)
* HOMEPAGE:
- * http://www.bouncycastle.org/
+ * https://www.bouncycastle.org/
-This product optionally depends on 'SLF4J', a simple logging facade for Java,
-which can be obtained at:
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
* LICENSE:
- * license/LICENSE.slf4j.txt (MIT License)
+ * license/LICENSE.snappy.txt (New BSD License)
* HOMEPAGE:
- * http://www.slf4j.org/
+ * https://github.com/google/snappy
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jboss-marshalling.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/jboss-remoting/jboss-marshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.caliper.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/google/caliper
This product optionally depends on 'Apache Commons Logging', a logging
framework, which can be obtained at:
@@ -79,61 +178,79 @@ framework, which can be obtained at:
* LICENSE:
* license/LICENSE.commons-logging.txt (Apache License 2.0)
* HOMEPAGE:
- * http://commons.apache.org/logging/
+ * https://commons.apache.org/logging/
-This product optionally depends on 'Apache Logback', a logging framework,
-which can be obtained at:
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
* LICENSE:
- * license/LICENSE.logback.txt (Eclipse Public License 1.0)
+ * license/LICENSE.log4j.txt (Apache License 2.0)
* HOMEPAGE:
- * https://logback.qos.ch/
+ * https://logging.apache.org/log4j/
-This product optionally depends on 'JBoss Logging', a logging framework,
-which can be obtained at:
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
* LICENSE:
- * license/LICENSE.jboss-logging.txt (GNU LGPL 2.1)
+ * license/LICENSE.aalto-xml.txt (Apache License 2.0)
* HOMEPAGE:
- * http://anonsvn.jboss.org/repos/common/common-logging-spi/
+ * https://wiki.fasterxml.com/AaltoHome
-This product optionally depends on 'Apache Felix', an open source OSGi
-framework implementation, which can be obtained at:
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
* LICENSE:
- * license/LICENSE.felix.txt (Apache License 2.0)
+ * license/LICENSE.hpack.txt (Apache License 2.0)
* HOMEPAGE:
- * http://felix.apache.org/
+ * https://github.com/twitter/hpack
-The bundled library Metrics Core NOTICE file reports the following items
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Cory Benfield. It can be obtained at:
-Metrics
-Copyright 2010-2013 Coda Hale and Yammer, Inc.
+ * LICENSE:
+ * license/LICENSE.hyper-hpack.txt (MIT License)
+ * HOMEPAGE:
+ * https://github.com/python-hyper/hpack/
-This product includes software developed by Coda Hale and Yammer, Inc.
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Tatsuhiro Tsujikawa. It can be obtained at:
-This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
-LongAdder), which was released with the following comments:
+ * LICENSE:
+ * license/LICENSE.nghttp2-hpack.txt (MIT License)
+ * HOMEPAGE:
+ * https://github.com/nghttp2/nghttp2/
- Written by Doug Lea with assistance from members of JCP JSR-166
- Expert Group and released to the public domain, as explained at
- http://creativecommons.org/publicdomain/zero/1.0/
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
-The Nappy Java NOTICE file reports the following items:
+ * LICENSE:
+ * license/LICENSE.commons-lang.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://commons.apache.org/proper/commons-lang/
-This product includes software developed by Google
- Snappy: http://code.google.com/p/snappy/ (New BSD License)
-This product includes software developed by Apache
- PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/
- (Apache 2.0 license)
+This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build.
-This library containd statically linked libstdc++. This inclusion is allowed by
-"GCC RUntime Library Exception"
-http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
+ * LICENSE:
+ * license/LICENSE.mvn-wrapper.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/takari/maven-wrapper
+
+This product contains the dnsinfo.h header file, that provides a way to retrieve the system DNS configuration on MacOS.
+This private header is also used by Apple's open source
+ mDNSResponder (https://opensource.apple.com/tarballs/mDNSResponder/).
+
+ * LICENSE:
+ * license/LICENSE.dnsinfo.txt (Apple Public Source License 2.0)
+ * HOMEPAGE:
+ * https://www.opensource.apple.com/source/configd/configd-453.19/dnsinfo/dnsinfo.h
+
+This product optionally depends on 'Brotli4j', Brotli compression and
+decompression for Java., which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.brotli4j.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/hyperxpro/Brotli4j
-== Contributors ==
- * Tatu Saloranta
- * Providing benchmark suite
- * Alec Wysoker
- * Performance and memory usage improvement
+---------------- end of netty NOTICE file ----------------
diff --git a/zookeeper-server/src/main/resources/lib/commons-io-2.7.LICENSE.txt b/zookeeper-server/src/main/resources/lib/commons-cli-1.5.0.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/commons-io-2.7.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/commons-cli-1.5.0.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/netty-handler-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/commons-io-2.11.0.LICENSE.txt
similarity index 99%
rename from zookeeper-server/src/main/resources/lib/netty-handler-4.1.73.Final.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/commons-io-2.11.0.LICENSE.txt
index 6279e5206de..6b0b1270ff0 100644
--- a/zookeeper-server/src/main/resources/lib/netty-handler-4.1.73.Final.LICENSE.txt
+++ b/zookeeper-server/src/main/resources/lib/commons-io-2.11.0.LICENSE.txt
@@ -187,7 +187,7 @@
same "printed page" as the copyright notice for easier
identification within third-party archives.
- Copyright 1999-2005 The Apache Software Foundation
+ 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.
@@ -200,3 +200,4 @@
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/zookeeper-server/src/main/resources/lib/jetty-http-9.4.43.v20210629.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-client-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/jetty-http-9.4.43.v20210629.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-client-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/jetty-io-9.4.43.v20210629.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-http-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/jetty-io-9.4.43.v20210629.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-http-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/jetty-security-9.4.43.v20210629.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-io-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/jetty-security-9.4.43.v20210629.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-io-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/jetty-server-9.4.43.v20210629.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-security-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/jetty-server-9.4.43.v20210629.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-security-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/jetty-servlet-9.4.43.v20210629.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-server-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/jetty-servlet-9.4.43.v20210629.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-server-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/jetty-util-9.4.43.v20210629.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-servlet-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/jetty-util-9.4.43.v20210629.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-servlet-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/jetty-util-ajax-9.4.43.v20210629.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-util-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/jetty-util-ajax-9.4.43.v20210629.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-util-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/logback-classic-1.2.10.LICENSE.txt b/zookeeper-server/src/main/resources/lib/jetty-util-ajax-9.4.53.v20231009.LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/logback-classic-1.2.10.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/jetty-util-ajax-9.4.53.v20231009.LICENSE.txt
diff --git a/zookeeper-server/src/main/resources/lib/logback-classic-1.2.13.LICENSE.txt b/zookeeper-server/src/main/resources/lib/logback-classic-1.2.13.LICENSE.txt
new file mode 100644
index 00000000000..7e7bfbfc96e
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/logback-classic-1.2.13.LICENSE.txt
@@ -0,0 +1,688 @@
+Logback LICENSE
+---------------
+
+Logback: the reliable, generic, fast and flexible logging framework.
+Copyright (C) 1999-2015, QOS.ch. All rights reserved.
+
+This program and the accompanying materials are dual-licensed under
+either the terms of the Eclipse Public License v1.0 as published by
+the Eclipse Foundation
+
+ or (per the licensee's choosing)
+
+under the terms of the GNU Lesser General Public License version 2.1
+as published by the Free Software Foundation.
+
+
+
+Eclipse Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+ 1. DEFINITIONS
+
+ "Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation
+distributed under this Agreement, and
+
+ b) in the case of each subsequent Contributor:
+
+ i) changes to the Program, and
+
+ ii) additions to the Program;
+
+where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from
+a Contributor if it was added to the Program by such Contributor itself or
+anyone acting on such Contributor's behalf. Contributions do not include additions
+to the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are
+not derivative works of the Program.
+
+ "Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents" mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement, including
+all Contributors.
+
+ 2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce,
+prepare derivative works of, publicly display, publicly perform, distribute
+and sublicense the Contribution of such Contributor, if any, and such derivative
+works, in source code and object code form.
+
+b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
+Patents to make, use, sell, offer to sell, import and otherwise transfer the
+Contribution of such Contributor, if any, in source code and object code form.
+This patent license shall apply to the combination of the Contribution and
+the Program if, at the time the Contribution is added by the Contributor,
+such addition of the Contribution causes such combination to be covered by
+the Licensed Patents. The patent license shall not apply to any other combinations
+which include the Contribution. No hardware per se is licensed hereunder.
+
+c) Recipient understands that although each Contributor grants the licenses
+to its Contributions set forth herein, no assurances are provided by any Contributor
+that the Program does not infringe the patent or other intellectual property
+rights of any other entity. Each Contributor disclaims any liability to Recipient
+for claims brought by any other entity based on infringement of intellectual
+property rights or otherwise. As a condition to exercising the rights and
+licenses granted hereunder, each Recipient hereby assumes sole responsibility
+to secure any other intellectual property rights needed, if any. For example,
+if a third party patent license is required to allow Recipient to distribute
+the Program, it is Recipient's responsibility to acquire that license before
+distributing the Program.
+
+d) Each Contributor represents that to its knowledge it has sufficient copyright
+rights in its Contribution, if any, to grant the copyright license set forth
+in this Agreement.
+
+ 3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under
+its own license agreement, provided that:
+
+ a) it complies with the terms and conditions of this Agreement; and
+
+ b) its license agreement:
+
+i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title
+and non-infringement, and implied warranties or conditions of merchantability
+and fitness for a particular purpose;
+
+ii) effectively excludes on behalf of all Contributors all liability for damages,
+including direct, indirect, special, incidental and consequential damages,
+such as lost profits;
+
+iii) states that any provisions which differ from this Agreement are offered
+by that Contributor alone and not by any other party; and
+
+iv) states that source code for the Program is available from such Contributor,
+and informs licensees how to obtain it in a reasonable manner on or through
+a medium customarily used for software exchange.
+
+ When the Program is made available in source code form:
+
+ a) it must be made available under this Agreement; and
+
+b) a copy of this Agreement must be included with each copy of the Program.
+
+Contributors may not remove or alter any copyright notices contained within
+the Program.
+
+Each Contributor must identify itself as the originator of its Contribution,
+if any, in a manner that reasonably allows subsequent Recipients to identify
+the originator of the Contribution.
+
+ 4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor
+who includes the Program in a commercial product offering should do so in
+a manner which does not create potential liability for other Contributors.
+Therefore, if a Contributor includes the Program in a commercial product offering,
+such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other
+legal actions brought by a third party against the Indemnified Contributor
+to the extent caused by the acts or omissions of such Commercial Contributor
+in connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor
+to control, and cooperate with the Commercial Contributor in, the defense
+and any related settlement negotiations. The Indemnified Contributor may participate
+in any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If
+that Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such Commercial
+Contributor's responsibility alone. Under this section, the Commercial Contributor
+would have to defend claims against the other Contributors related to those
+performance claims and warranties, and if a court requires any other Contributor
+to pay any damages as a result, the Commercial Contributor must pay those
+damages.
+
+ 5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED 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.
+Each Recipient is solely responsible for determining the appropriateness of
+using and distributing the Program and assumes all risks associated with its
+exercise of rights under this Agreement, including but not limited to the
+risks and costs of program errors, compliance with applicable laws, damage
+to or loss of data, programs or equipment, and unavailability or interruption
+of operations.
+
+ 6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION
+LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
+WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+ 7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of
+the terms of this Agreement, and without further action by the parties hereto,
+such provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a
+cross-claim or counterclaim in a lawsuit) alleging that the Program itself
+(excluding combinations of the Program with other software or hardware) infringes
+such Recipient's patent(s), then such Recipient's rights granted under Section
+2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and
+does not cure such failure in a reasonable period of time after becoming aware
+of such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as reasonably
+practicable. However, Recipient's obligations under this Agreement and any
+licenses granted by Recipient relating to the Program shall continue and survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but
+in order to avoid inconsistency the Agreement is copyrighted and may only
+be modified in the following manner. The Agreement Steward reserves the right
+to publish new versions (including revisions) of this Agreement from time
+to time. No one other than the Agreement Steward has the right to modify this
+Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse
+Foundation may assign the responsibility to serve as the Agreement Steward
+to a suitable separate entity. Each new version of the Agreement will be given
+a distinguishing version number. The Program (including Contributions) may
+always be distributed subject to the version of the Agreement under which
+it was received. In addition, after a new version of the Agreement is published,
+Contributor may elect to distribute the Program (including its Contributions)
+under the new version. Except as expressly stated in Sections 2(a) and 2(b)
+above, Recipient receives no rights or licenses to the intellectual property
+of any Contributor under this Agreement, whether expressly, by implication,
+estoppel or otherwise. All rights in the Program not expressly granted under
+this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the intellectual
+property laws of the United States of America. No party to this Agreement
+will bring a legal action under this Agreement more than one year after the
+cause of action arose. Each party waives its rights to a jury trial in any
+resulting litigation.
+
+
+
+ GNU LESSER GENERAL PUBLIC LICENSE
+ Version 2.1, February 1999
+
+ Copyright (C) 1991, 1999 Free Software Foundation, Inc.
+ 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
+ Everyone is permitted to copy and distribute verbatim copies
+ of this license document, but changing it is not allowed.
+
+[This is the first released version of the Lesser GPL. It also counts
+ as the successor of the GNU Library Public License, version 2, hence
+ the version number 2.1.]
+
+ Preamble
+
+ The licenses for most software are designed to take away your
+freedom to share and change it. By contrast, the GNU General Public
+Licenses are intended to guarantee your freedom to share and change
+free software--to make sure the software is free for all its users.
+
+ This license, the Lesser General Public License, applies to some
+specially designated software packages--typically libraries--of the
+Free Software Foundation and other authors who decide to use it. You
+can use it too, but we suggest you first think carefully about whether
+this license or the ordinary General Public License is the better
+strategy to use in any particular case, based on the explanations below.
+
+ When we speak of free software, we are referring to freedom of use,
+not price. Our General Public Licenses are designed to make sure that
+you have the freedom to distribute copies of free software (and charge
+for this service if you wish); that you receive source code or can get
+it if you want it; that you can change the software and use pieces of
+it in new free programs; and that you are informed that you can do
+these things.
+
+ To protect your rights, we need to make restrictions that forbid
+distributors to deny you these rights or to ask you to surrender these
+rights. These restrictions translate to certain responsibilities for
+you if you distribute copies of the library or if you modify it.
+
+ For example, if you distribute copies of the library, whether gratis
+or for a fee, you must give the recipients all the rights that we gave
+you. You must make sure that they, too, receive or can get the source
+code. If you link other code with the library, you must provide
+complete object files to the recipients, so that they can relink them
+with the library after making changes to the library and recompiling
+it. And you must show them these terms so they know their rights.
+
+ We protect your rights with a two-step method: (1) we copyright the
+library, and (2) we offer you this license, which gives you legal
+permission to copy, distribute and/or modify the library.
+
+ To protect each distributor, we want to make it very clear that
+there is no warranty for the free library. Also, if the library is
+modified by someone else and passed on, the recipients should know
+that what they have is not the original version, so that the original
+author's reputation will not be affected by problems that might be
+introduced by others.
+
+ Finally, software patents pose a constant threat to the existence of
+any free program. We wish to make sure that a company cannot
+effectively restrict the users of a free program by obtaining a
+restrictive license from a patent holder. Therefore, we insist that
+any patent license obtained for a version of the library must be
+consistent with the full freedom of use specified in this license.
+
+ Most GNU software, including some libraries, is covered by the
+ordinary GNU General Public License. This license, the GNU Lesser
+General Public License, applies to certain designated libraries, and
+is quite different from the ordinary General Public License. We use
+this license for certain libraries in order to permit linking those
+libraries into non-free programs.
+
+ When a program is linked with a library, whether statically or using
+a shared library, the combination of the two is legally speaking a
+combined work, a derivative of the original library. The ordinary
+General Public License therefore permits such linking only if the
+entire combination fits its criteria of freedom. The Lesser General
+Public License permits more lax criteria for linking other code with
+the library.
+
+ We call this license the "Lesser" General Public License because it
+does Less to protect the user's freedom than the ordinary General
+Public License. It also provides other free software developers Less
+of an advantage over competing non-free programs. These disadvantages
+are the reason we use the ordinary General Public License for many
+libraries. However, the Lesser license provides advantages in certain
+special circumstances.
+
+ For example, on rare occasions, there may be a special need to
+encourage the widest possible use of a certain library, so that it becomes
+a de-facto standard. To achieve this, non-free programs must be
+allowed to use the library. A more frequent case is that a free
+library does the same job as widely used non-free libraries. In this
+case, there is little to gain by limiting the free library to free
+software only, so we use the Lesser General Public License.
+
+ In other cases, permission to use a particular library in non-free
+programs enables a greater number of people to use a large body of
+free software. For example, permission to use the GNU C Library in
+non-free programs enables many more people to use the whole GNU
+operating system, as well as its variant, the GNU/Linux operating
+system.
+
+ Although the Lesser General Public License is Less protective of the
+users' freedom, it does ensure that the user of a program that is
+linked with the Library has the freedom and the wherewithal to run
+that program using a modified version of the Library.
+
+ The precise terms and conditions for copying, distribution and
+modification follow. Pay close attention to the difference between a
+"work based on the library" and a "work that uses the library". The
+former contains code derived from the library, whereas the latter must
+be combined with the library in order to run.
+
+ GNU LESSER GENERAL PUBLIC LICENSE
+ TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+ 0. This License Agreement applies to any software library or other
+program which contains a notice placed by the copyright holder or
+other authorized party saying it may be distributed under the terms of
+this Lesser General Public License (also called "this License").
+Each licensee is addressed as "you".
+
+ A "library" means a collection of software functions and/or data
+prepared so as to be conveniently linked with application programs
+(which use some of those functions and data) to form executables.
+
+ The "Library", below, refers to any such software library or work
+which has been distributed under these terms. A "work based on the
+Library" means either the Library or any derivative work under
+copyright law: that is to say, a work containing the Library or a
+portion of it, either verbatim or with modifications and/or translated
+straightforwardly into another language. (Hereinafter, translation is
+included without limitation in the term "modification".)
+
+ "Source code" for a work means the preferred form of the work for
+making modifications to it. For a library, complete source code means
+all the source code for all modules it contains, plus any associated
+interface definition files, plus the scripts used to control compilation
+and installation of the library.
+
+ Activities other than copying, distribution and modification are not
+covered by this License; they are outside its scope. The act of
+running a program using the Library is not restricted, and output from
+such a program is covered only if its contents constitute a work based
+on the Library (independent of the use of the Library in a tool for
+writing it). Whether that is true depends on what the Library does
+and what the program that uses the Library does.
+
+ 1. You may copy and distribute verbatim copies of the Library's
+complete source code as you receive it, in any medium, provided that
+you conspicuously and appropriately publish on each copy an
+appropriate copyright notice and disclaimer of warranty; keep intact
+all the notices that refer to this License and to the absence of any
+warranty; and distribute a copy of this License along with the
+Library.
+
+ You may charge a fee for the physical act of transferring a copy,
+and you may at your option offer warranty protection in exchange for a
+fee.
+
+ 2. You may modify your copy or copies of the Library or any portion
+of it, thus forming a work based on the Library, and copy and
+distribute such modifications or work under the terms of Section 1
+above, provided that you also meet all of these conditions:
+
+ a) The modified work must itself be a software library.
+
+ b) You must cause the files modified to carry prominent notices
+ stating that you changed the files and the date of any change.
+
+ c) You must cause the whole of the work to be licensed at no
+ charge to all third parties under the terms of this License.
+
+ d) If a facility in the modified Library refers to a function or a
+ table of data to be supplied by an application program that uses
+ the facility, other than as an argument passed when the facility
+ is invoked, then you must make a good faith effort to ensure that,
+ in the event an application does not supply such function or
+ table, the facility still operates, and performs whatever part of
+ its purpose remains meaningful.
+
+ (For example, a function in a library to compute square roots has
+ a purpose that is entirely well-defined independent of the
+ application. Therefore, Subsection 2d requires that any
+ application-supplied function or table used by this function must
+ be optional: if the application does not supply it, the square
+ root function must still compute square roots.)
+
+These requirements apply to the modified work as a whole. If
+identifiable sections of that work are not derived from the Library,
+and can be reasonably considered independent and separate works in
+themselves, then this License, and its terms, do not apply to those
+sections when you distribute them as separate works. But when you
+distribute the same sections as part of a whole which is a work based
+on the Library, the distribution of the whole must be on the terms of
+this License, whose permissions for other licensees extend to the
+entire whole, and thus to each and every part regardless of who wrote
+it.
+
+Thus, it is not the intent of this section to claim rights or contest
+your rights to work written entirely by you; rather, the intent is to
+exercise the right to control the distribution of derivative or
+collective works based on the Library.
+
+In addition, mere aggregation of another work not based on the Library
+with the Library (or with a work based on the Library) on a volume of
+a storage or distribution medium does not bring the other work under
+the scope of this License.
+
+ 3. You may opt to apply the terms of the ordinary GNU General Public
+License instead of this License to a given copy of the Library. To do
+this, you must alter all the notices that refer to this License, so
+that they refer to the ordinary GNU General Public License, version 2,
+instead of to this License. (If a newer version than version 2 of the
+ordinary GNU General Public License has appeared, then you can specify
+that version instead if you wish.) Do not make any other change in
+these notices.
+
+ Once this change is made in a given copy, it is irreversible for
+that copy, so the ordinary GNU General Public License applies to all
+subsequent copies and derivative works made from that copy.
+
+ This option is useful when you wish to copy part of the code of
+the Library into a program that is not a library.
+
+ 4. You may copy and distribute the Library (or a portion or
+derivative of it, under Section 2) in object code or executable form
+under the terms of Sections 1 and 2 above provided that you accompany
+it with the complete corresponding machine-readable source code, which
+must be distributed under the terms of Sections 1 and 2 above on a
+medium customarily used for software interchange.
+
+ If distribution of object code is made by offering access to copy
+from a designated place, then offering equivalent access to copy the
+source code from the same place satisfies the requirement to
+distribute the source code, even though third parties are not
+compelled to copy the source along with the object code.
+
+ 5. A program that contains no derivative of any portion of the
+Library, but is designed to work with the Library by being compiled or
+linked with it, is called a "work that uses the Library". Such a
+work, in isolation, is not a derivative work of the Library, and
+therefore falls outside the scope of this License.
+
+ However, linking a "work that uses the Library" with the Library
+creates an executable that is a derivative of the Library (because it
+contains portions of the Library), rather than a "work that uses the
+library". The executable is therefore covered by this License.
+Section 6 states terms for distribution of such executables.
+
+ When a "work that uses the Library" uses material from a header file
+that is part of the Library, the object code for the work may be a
+derivative work of the Library even though the source code is not.
+Whether this is true is especially significant if the work can be
+linked without the Library, or if the work is itself a library. The
+threshold for this to be true is not precisely defined by law.
+
+ If such an object file uses only numerical parameters, data
+structure layouts and accessors, and small macros and small inline
+functions (ten lines or less in length), then the use of the object
+file is unrestricted, regardless of whether it is legally a derivative
+work. (Executables containing this object code plus portions of the
+Library will still fall under Section 6.)
+
+ Otherwise, if the work is a derivative of the Library, you may
+distribute the object code for the work under the terms of Section 6.
+Any executables containing that work also fall under Section 6,
+whether or not they are linked directly with the Library itself.
+
+ 6. As an exception to the Sections above, you may also combine or
+link a "work that uses the Library" with the Library to produce a
+work containing portions of the Library, and distribute that work
+under terms of your choice, provided that the terms permit
+modification of the work for the customer's own use and reverse
+engineering for debugging such modifications.
+
+ You must give prominent notice with each copy of the work that the
+Library is used in it and that the Library and its use are covered by
+this License. You must supply a copy of this License. If the work
+during execution displays copyright notices, you must include the
+copyright notice for the Library among them, as well as a reference
+directing the user to the copy of this License. Also, you must do one
+of these things:
+
+ a) Accompany the work with the complete corresponding
+ machine-readable source code for the Library including whatever
+ changes were used in the work (which must be distributed under
+ Sections 1 and 2 above); and, if the work is an executable linked
+ with the Library, with the complete machine-readable "work that
+ uses the Library", as object code and/or source code, so that the
+ user can modify the Library and then relink to produce a modified
+ executable containing the modified Library. (It is understood
+ that the user who changes the contents of definitions files in the
+ Library will not necessarily be able to recompile the application
+ to use the modified definitions.)
+
+ b) Use a suitable shared library mechanism for linking with the
+ Library. A suitable mechanism is one that (1) uses at run time a
+ copy of the library already present on the user's computer system,
+ rather than copying library functions into the executable, and (2)
+ will operate properly with a modified version of the library, if
+ the user installs one, as long as the modified version is
+ interface-compatible with the version that the work was made with.
+
+ c) Accompany the work with a written offer, valid for at
+ least three years, to give the same user the materials
+ specified in Subsection 6a, above, for a charge no more
+ than the cost of performing this distribution.
+
+ d) If distribution of the work is made by offering access to copy
+ from a designated place, offer equivalent access to copy the above
+ specified materials from the same place.
+
+ e) Verify that the user has already received a copy of these
+ materials or that you have already sent this user a copy.
+
+ For an executable, the required form of the "work that uses the
+Library" must include any data and utility programs needed for
+reproducing the executable from it. However, as a special exception,
+the materials to be distributed need not include anything that is
+normally distributed (in either source or binary form) with the major
+components (compiler, kernel, and so on) of the operating system on
+which the executable runs, unless that component itself accompanies
+the executable.
+
+ It may happen that this requirement contradicts the license
+restrictions of other proprietary libraries that do not normally
+accompany the operating system. Such a contradiction means you cannot
+use both them and the Library together in an executable that you
+distribute.
+
+ 7. You may place library facilities that are a work based on the
+Library side-by-side in a single library together with other library
+facilities not covered by this License, and distribute such a combined
+library, provided that the separate distribution of the work based on
+the Library and of the other library facilities is otherwise
+permitted, and provided that you do these two things:
+
+ a) Accompany the combined library with a copy of the same work
+ based on the Library, uncombined with any other library
+ facilities. This must be distributed under the terms of the
+ Sections above.
+
+ b) Give prominent notice with the combined library of the fact
+ that part of it is a work based on the Library, and explaining
+ where to find the accompanying uncombined form of the same work.
+
+ 8. You may not copy, modify, sublicense, link with, or distribute
+the Library except as expressly provided under this License. Any
+attempt otherwise to copy, modify, sublicense, link with, or
+distribute the Library is void, and will automatically terminate your
+rights under this License. However, parties who have received copies,
+or rights, from you under this License will not have their licenses
+terminated so long as such parties remain in full compliance.
+
+ 9. You are not required to accept this License, since you have not
+signed it. However, nothing else grants you permission to modify or
+distribute the Library or its derivative works. These actions are
+prohibited by law if you do not accept this License. Therefore, by
+modifying or distributing the Library (or any work based on the
+Library), you indicate your acceptance of this License to do so, and
+all its terms and conditions for copying, distributing or modifying
+the Library or works based on it.
+
+ 10. Each time you redistribute the Library (or any work based on the
+Library), the recipient automatically receives a license from the
+original licensor to copy, distribute, link with or modify the Library
+subject to these terms and conditions. You may not impose any further
+restrictions on the recipients' exercise of the rights granted herein.
+You are not responsible for enforcing compliance by third parties with
+this License.
+
+ 11. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues),
+conditions are imposed on you (whether by court order, agreement or
+otherwise) that contradict the conditions of this License, they do not
+excuse you from the conditions of this License. If you cannot
+distribute so as to satisfy simultaneously your obligations under this
+License and any other pertinent obligations, then as a consequence you
+may not distribute the Library at all. For example, if a patent
+license would not permit royalty-free redistribution of the Library by
+all those who receive copies directly or indirectly through you, then
+the only way you could satisfy both it and this License would be to
+refrain entirely from distribution of the Library.
+
+If any portion of this section is held invalid or unenforceable under any
+particular circumstance, the balance of the section is intended to apply,
+and the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any
+patents or other property right claims or to contest validity of any
+such claims; this section has the sole purpose of protecting the
+integrity of the free software distribution system which is
+implemented by public license practices. Many people have made
+generous contributions to the wide range of software distributed
+through that system in reliance on consistent application of that
+system; it is up to the author/donor to decide if he or she is willing
+to distribute software through any other system and a licensee cannot
+impose that choice.
+
+This section is intended to make thoroughly clear what is believed to
+be a consequence of the rest of this License.
+
+ 12. If the distribution and/or use of the Library is restricted in
+certain countries either by patents or by copyrighted interfaces, the
+original copyright holder who places the Library under this License may add
+an explicit geographical distribution limitation excluding those countries,
+so that distribution is permitted only in or among countries not thus
+excluded. In such case, this License incorporates the limitation as if
+written in the body of this License.
+
+ 13. The Free Software Foundation may publish revised and/or new
+versions of the Lesser General Public License from time to time.
+Such new versions will be similar in spirit to the present version,
+but may differ in detail to address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Library
+specifies a version number of this License which applies to it and
+"any later version", you have the option of following the terms and
+conditions either of that version or of any later version published by
+the Free Software Foundation. If the Library does not specify a
+license version number, you may choose any version ever published by
+the Free Software Foundation.
+
+ 14. If you wish to incorporate parts of the Library into other free
+programs whose distribution conditions are incompatible with these,
+write to the author to ask for permission. For software which is
+copyrighted by the Free Software Foundation, write to the Free
+Software Foundation; we sometimes make exceptions for this. Our
+decision will be guided by the two goals of preserving the free status
+of all derivatives of our free software and of promoting the sharing
+and reuse of software generally.
+
+ NO WARRANTY
+
+ 15. BECAUSE THE LIBRARY IS LICENSED FREE OF CHARGE, THERE IS NO
+WARRANTY FOR THE LIBRARY, TO THE EXTENT PERMITTED BY APPLICABLE LAW.
+EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR
+OTHER PARTIES PROVIDE THE LIBRARY "AS IS" WITHOUT WARRANTY OF ANY
+KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE
+LIBRARY IS WITH YOU. SHOULD THE LIBRARY PROVE DEFECTIVE, YOU ASSUME
+THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+ 16. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN
+WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY
+AND/OR REDISTRIBUTE THE LIBRARY AS PERMITTED ABOVE, BE LIABLE TO YOU
+FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR
+CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE
+LIBRARY (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING
+RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A
+FAILURE OF THE LIBRARY TO OPERATE WITH ANY OTHER SOFTWARE), EVEN IF
+SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH
+DAMAGES.
+
+ END OF TERMS AND CONDITIONS
diff --git a/zookeeper-server/src/main/resources/lib/logback-core-1.2.10.LICENSE.txt b/zookeeper-server/src/main/resources/lib/logback-core-1.2.10.LICENSE.txt
deleted file mode 100644
index 46f4f252464..00000000000
--- a/zookeeper-server/src/main/resources/lib/logback-core-1.2.10.LICENSE.txt
+++ /dev/null
@@ -1,414 +0,0 @@
-This program and the accompanying materials are made available under the
-terms of the Eclipse Public License 2.0 which is available at
-http://www.eclipse.org/legal/epl-2.0, or the Apache Software License
-2.0 which is available at https://www.apache.org/licenses/LICENSE-2.0.
-
-
-
-Eclipse Public License - v 1.0
-
-THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
-LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
-CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
-
-1. DEFINITIONS
-
-"Contribution" means:
-
-a) in the case of the initial Contributor, the initial code and documentation
- distributed under this Agreement, and
-b) in the case of each subsequent Contributor:
- i) changes to the Program, and
- ii) additions to the Program;
-
- where such changes and/or additions to the Program originate from and are
- distributed by that particular Contributor. A Contribution 'originates'
- from a Contributor if it was added to the Program by such Contributor
- itself or anyone acting on such Contributor's behalf. Contributions do not
- include additions to the Program which: (i) are separate modules of
- software distributed in conjunction with the Program under their own
- license agreement, and (ii) are not derivative works of the Program.
-
-"Contributor" means any person or entity that distributes the Program.
-
-"Licensed Patents" mean patent claims licensable by a Contributor which are
-necessarily infringed by the use or sale of its Contribution alone or when
-combined with the Program.
-
-"Program" means the Contributions distributed in accordance with this
-Agreement.
-
-"Recipient" means anyone who receives the Program under this Agreement,
-including all Contributors.
-
-2. GRANT OF RIGHTS
- a) Subject to the terms of this Agreement, each Contributor hereby grants
- Recipient a non-exclusive, worldwide, royalty-free copyright license to
- reproduce, prepare derivative works of, publicly display, publicly
- perform, distribute and sublicense the Contribution of such Contributor,
- if any, and such derivative works, in source code and object code form.
- b) Subject to the terms of this Agreement, each Contributor hereby grants
- Recipient a non-exclusive, worldwide, royalty-free patent license under
- Licensed Patents to make, use, sell, offer to sell, import and otherwise
- transfer the Contribution of such Contributor, if any, in source code and
- object code form. This patent license shall apply to the combination of
- the Contribution and the Program if, at the time the Contribution is
- added by the Contributor, such addition of the Contribution causes such
- combination to be covered by the Licensed Patents. The patent license
- shall not apply to any other combinations which include the Contribution.
- No hardware per se is licensed hereunder.
- c) Recipient understands that although each Contributor grants the licenses
- to its Contributions set forth herein, no assurances are provided by any
- Contributor that the Program does not infringe the patent or other
- intellectual property rights of any other entity. Each Contributor
- disclaims any liability to Recipient for claims brought by any other
- entity based on infringement of intellectual property rights or
- otherwise. As a condition to exercising the rights and licenses granted
- hereunder, each Recipient hereby assumes sole responsibility to secure
- any other intellectual property rights needed, if any. For example, if a
- third party patent license is required to allow Recipient to distribute
- the Program, it is Recipient's responsibility to acquire that license
- before distributing the Program.
- d) Each Contributor represents that to its knowledge it has sufficient
- copyright rights in its Contribution, if any, to grant the copyright
- license set forth in this Agreement.
-
-3. REQUIREMENTS
-
-A Contributor may choose to distribute the Program in object code form under
-its own license agreement, provided that:
-
- a) it complies with the terms and conditions of this Agreement; and
- b) its license agreement:
- i) effectively disclaims on behalf of all Contributors all warranties
- and conditions, express and implied, including warranties or
- conditions of title and non-infringement, and implied warranties or
- conditions of merchantability and fitness for a particular purpose;
- ii) effectively excludes on behalf of all Contributors all liability for
- damages, including direct, indirect, special, incidental and
- consequential damages, such as lost profits;
- iii) states that any provisions which differ from this Agreement are
- offered by that Contributor alone and not by any other party; and
- iv) states that source code for the Program is available from such
- Contributor, and informs licensees how to obtain it in a reasonable
- manner on or through a medium customarily used for software exchange.
-
-When the Program is made available in source code form:
-
- a) it must be made available under this Agreement; and
- b) a copy of this Agreement must be included with each copy of the Program.
- Contributors may not remove or alter any copyright notices contained
- within the Program.
-
-Each Contributor must identify itself as the originator of its Contribution,
-if
-any, in a manner that reasonably allows subsequent Recipients to identify the
-originator of the Contribution.
-
-4. COMMERCIAL DISTRIBUTION
-
-Commercial distributors of software may accept certain responsibilities with
-respect to end users, business partners and the like. While this license is
-intended to facilitate the commercial use of the Program, the Contributor who
-includes the Program in a commercial product offering should do so in a manner
-which does not create potential liability for other Contributors. Therefore,
-if a Contributor includes the Program in a commercial product offering, such
-Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
-every other Contributor ("Indemnified Contributor") against any losses,
-damages and costs (collectively "Losses") arising from claims, lawsuits and
-other legal actions brought by a third party against the Indemnified
-Contributor to the extent caused by the acts or omissions of such Commercial
-Contributor in connection with its distribution of the Program in a commercial
-product offering. The obligations in this section do not apply to any claims
-or Losses relating to any actual or alleged intellectual property
-infringement. In order to qualify, an Indemnified Contributor must:
-a) promptly notify the Commercial Contributor in writing of such claim, and
-b) allow the Commercial Contributor to control, and cooperate with the
-Commercial Contributor in, the defense and any related settlement
-negotiations. The Indemnified Contributor may participate in any such claim at
-its own expense.
-
-For example, a Contributor might include the Program in a commercial product
-offering, Product X. That Contributor is then a Commercial Contributor. If
-that Commercial Contributor then makes performance claims, or offers
-warranties related to Product X, those performance claims and warranties are
-such Commercial Contributor's responsibility alone. Under this section, the
-Commercial Contributor would have to defend claims against the other
-Contributors related to those performance claims and warranties, and if a
-court requires any other Contributor to pay any damages as a result, the
-Commercial Contributor must pay those damages.
-
-5. NO WARRANTY
-
-EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED 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. Each
-Recipient is solely responsible for determining the appropriateness of using
-and distributing the Program and assumes all risks associated with its
-exercise of rights under this Agreement , including but not limited to the
-risks and costs of program errors, compliance with applicable laws, damage to
-or loss of data, programs or equipment, and unavailability or interruption of
-operations.
-
-6. DISCLAIMER OF LIABILITY
-
-EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
-CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION
-LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE
-EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY
-OF SUCH DAMAGES.
-
-7. GENERAL
-
-If any provision of this Agreement is invalid or unenforceable under
-applicable law, it shall not affect the validity or enforceability of the
-remainder of the terms of this Agreement, and without further action by the
-parties hereto, such provision shall be reformed to the minimum extent
-necessary to make such provision valid and enforceable.
-
-If Recipient institutes patent litigation against any entity (including a
-cross-claim or counterclaim in a lawsuit) alleging that the Program itself
-(excluding combinations of the Program with other software or hardware)
-infringes such Recipient's patent(s), then such Recipient's rights granted
-under Section 2(b) shall terminate as of the date such litigation is filed.
-
-All Recipient's rights under this Agreement shall terminate if it fails to
-comply with any of the material terms or conditions of this Agreement and does
-not cure such failure in a reasonable period of time after becoming aware of
-such noncompliance. If all Recipient's rights under this Agreement terminate,
-Recipient agrees to cease use and distribution of the Program as soon as
-reasonably practicable. However, Recipient's obligations under this Agreement
-and any licenses granted by Recipient relating to the Program shall continue
-and survive.
-
-Everyone is permitted to copy and distribute copies of this Agreement, but in
-order to avoid inconsistency the Agreement is copyrighted and may only be
-modified in the following manner. The Agreement Steward reserves the right to
-publish new versions (including revisions) of this Agreement from time to
-time. No one other than the Agreement Steward has the right to modify this
-Agreement. The Eclipse Foundation is the initial Agreement Steward. The
-Eclipse Foundation may assign the responsibility to serve as the Agreement
-Steward to a suitable separate entity. Each new version of the Agreement will
-be given a distinguishing version number. The Program (including
-Contributions) may always be distributed subject to the version of the
-Agreement under which it was received. In addition, after a new version of the
-Agreement is published, Contributor may elect to distribute the Program
-(including its Contributions) under the new version. Except as expressly
-stated in Sections 2(a) and 2(b) above, Recipient receives no rights or
-licenses to the intellectual property of any Contributor under this Agreement,
-whether expressly, by implication, estoppel or otherwise. All rights in the
-Program not expressly granted under this Agreement are reserved.
-
-This Agreement is governed by the laws of the State of New York and the
-intellectual property laws of the United States of America. No party to this
-Agreement will bring a legal action under this Agreement more than one year
-after the cause of action arose. Each party waives its rights to a jury trial in
-any resulting litigation.
-
-
-
- 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.
diff --git a/zookeeper-server/src/main/resources/lib/logback-core-1.2.13.LICENSE.txt b/zookeeper-server/src/main/resources/lib/logback-core-1.2.13.LICENSE.txt
new file mode 100644
index 00000000000..7e7bfbfc96e
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/logback-core-1.2.13.LICENSE.txt
@@ -0,0 +1,688 @@
+Logback LICENSE
+---------------
+
+Logback: the reliable, generic, fast and flexible logging framework.
+Copyright (C) 1999-2015, QOS.ch. All rights reserved.
+
+This program and the accompanying materials are dual-licensed under
+either the terms of the Eclipse Public License v1.0 as published by
+the Eclipse Foundation
+
+ or (per the licensee's choosing)
+
+under the terms of the GNU Lesser General Public License version 2.1
+as published by the Free Software Foundation.
+
+
+
+Eclipse Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+ 1. DEFINITIONS
+
+ "Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation
+distributed under this Agreement, and
+
+ b) in the case of each subsequent Contributor:
+
+ i) changes to the Program, and
+
+ ii) additions to the Program;
+
+where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from
+a Contributor if it was added to the Program by such Contributor itself or
+anyone acting on such Contributor's behalf. Contributions do not include additions
+to the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are
+not derivative works of the Program.
+
+ "Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents" mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement, including
+all Contributors.
+
+ 2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce,
+prepare derivative works of, publicly display, publicly perform, distribute
+and sublicense the Contribution of such Contributor, if any, and such derivative
+works, in source code and object code form.
+
+b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
+Patents to make, use, sell, offer to sell, import and otherwise transfer the
+Contribution of such Contributor, if any, in source code and object code form.
+This patent license shall apply to the combination of the Contribution and
+the Program if, at the time the Contribution is added by the Contributor,
+such addition of the Contribution causes such combination to be covered by
+the Licensed Patents. The patent license shall not apply to any other combinations
+which include the Contribution. No hardware per se is licensed hereunder.
+
+c) Recipient understands that although each Contributor grants the licenses
+to its Contributions set forth herein, no assurances are provided by any Contributor
+that the Program does not infringe the patent or other intellectual property
+rights of any other entity. Each Contributor disclaims any liability to Recipient
+for claims brought by any other entity based on infringement of intellectual
+property rights or otherwise. As a condition to exercising the rights and
+licenses granted hereunder, each Recipient hereby assumes sole responsibility
+to secure any other intellectual property rights needed, if any. For example,
+if a third party patent license is required to allow Recipient to distribute
+the Program, it is Recipient's responsibility to acquire that license before
+distributing the Program.
+
+d) Each Contributor represents that to its knowledge it has sufficient copyright
+rights in its Contribution, if any, to grant the copyright license set forth
+in this Agreement.
+
+ 3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under
+its own license agreement, provided that:
+
+ a) it complies with the terms and conditions of this Agreement; and
+
+ b) its license agreement:
+
+i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title
+and non-infringement, and implied warranties or conditions of merchantability
+and fitness for a particular purpose;
+
+ii) effectively excludes on behalf of all Contributors all liability for damages,
+including direct, indirect, special, incidental and consequential damages,
+such as lost profits;
+
+iii) states that any provisions which differ from this Agreement are offered
+by that Contributor alone and not by any other party; and
+
+iv) states that source code for the Program is available from such Contributor,
+and informs licensees how to obtain it in a reasonable manner on or through
+a medium customarily used for software exchange.
+
+ When the Program is made available in source code form:
+
+ a) it must be made available under this Agreement; and
+
+b) a copy of this Agreement must be included with each copy of the Program.
+
+Contributors may not remove or alter any copyright notices contained within
+the Program.
+
+Each Contributor must identify itself as the originator of its Contribution,
+if any, in a manner that reasonably allows subsequent Recipients to identify
+the originator of the Contribution.
+
+ 4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor
+who includes the Program in a commercial product offering should do so in
+a manner which does not create potential liability for other Contributors.
+Therefore, if a Contributor includes the Program in a commercial product offering,
+such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other
+legal actions brought by a third party against the Indemnified Contributor
+to the extent caused by the acts or omissions of such Commercial Contributor
+in connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor
+to control, and cooperate with the Commercial Contributor in, the defense
+and any related settlement negotiations. The Indemnified Contributor may participate
+in any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If
+that Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such Commercial
+Contributor's responsibility alone. Under this section, the Commercial Contributor
+would have to defend claims against the other Contributors related to those
+performance claims and warranties, and if a court requires any other Contributor
+to pay any damages as a result, the Commercial Contributor must pay those
+damages.
+
+ 5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED 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.
+Each Recipient is solely responsible for determining the appropriateness of
+using and distributing the Program and assumes all risks associated with its
+exercise of rights under this Agreement, including but not limited to the
+risks and costs of program errors, compliance with applicable laws, damage
+to or loss of data, programs or equipment, and unavailability or interruption
+of operations.
+
+ 6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION
+LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
+WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+ 7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of
+the terms of this Agreement, and without further action by the parties hereto,
+such provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a
+cross-claim or counterclaim in a lawsuit) alleging that the Program itself
+(excluding combinations of the Program with other software or hardware) infringes
+such Recipient's patent(s), then such Recipient's rights granted under Section
+2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and
+does not cure such failure in a reasonable period of time after becoming aware
+of such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as reasonably
+practicable. However, Recipient's obligations under this Agreement and any
+licenses granted by Recipient relating to the Program shall continue and survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but
+in order to avoid inconsistency the Agreement is copyrighted and may only
+be modified in the following manner. The Agreement Steward reserves the right
+to publish new versions (including revisions) of this Agreement from time
+to time. No one other than the Agreement Steward has the right to modify this
+Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse
+Foundation may assign the responsibility to serve as the Agreement Steward
+to a suitable separate entity. Each new version of the Agreement will be given
+a distinguishing version number. The Program (including Contributions) may
+always be distributed subject to the version of the Agreement under which
+it was received. In addition, after a new version of the Agreement is published,
+Contributor may elect to distribute the Program (including its Contributions)
+under the new version. Except as expressly stated in Sections 2(a) and 2(b)
+above, Recipient receives no rights or licenses to the intellectual property
+of any Contributor under this Agreement, whether expressly, by implication,
+estoppel or otherwise. All rights in the Program not expressly granted under
+this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the intellectual
+property laws of the United States of America. No party to this Agreement
+will bring a legal action under this Agreement more than one year after the
+cause of action arose. Each party waives its rights to a jury trial in any
+resulting litigation.
+
+
+
+ GNU LESSER GENERAL PUBLIC LICENSE
+ Version 2.1, February 1999
+
+ Copyright (C) 1991, 1999 Free Software Foundation, Inc.
+ 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
+ Everyone is permitted to copy and distribute verbatim copies
+ of this license document, but changing it is not allowed.
+
+[This is the first released version of the Lesser GPL. It also counts
+ as the successor of the GNU Library Public License, version 2, hence
+ the version number 2.1.]
+
+ Preamble
+
+ The licenses for most software are designed to take away your
+freedom to share and change it. By contrast, the GNU General Public
+Licenses are intended to guarantee your freedom to share and change
+free software--to make sure the software is free for all its users.
+
+ This license, the Lesser General Public License, applies to some
+specially designated software packages--typically libraries--of the
+Free Software Foundation and other authors who decide to use it. You
+can use it too, but we suggest you first think carefully about whether
+this license or the ordinary General Public License is the better
+strategy to use in any particular case, based on the explanations below.
+
+ When we speak of free software, we are referring to freedom of use,
+not price. Our General Public Licenses are designed to make sure that
+you have the freedom to distribute copies of free software (and charge
+for this service if you wish); that you receive source code or can get
+it if you want it; that you can change the software and use pieces of
+it in new free programs; and that you are informed that you can do
+these things.
+
+ To protect your rights, we need to make restrictions that forbid
+distributors to deny you these rights or to ask you to surrender these
+rights. These restrictions translate to certain responsibilities for
+you if you distribute copies of the library or if you modify it.
+
+ For example, if you distribute copies of the library, whether gratis
+or for a fee, you must give the recipients all the rights that we gave
+you. You must make sure that they, too, receive or can get the source
+code. If you link other code with the library, you must provide
+complete object files to the recipients, so that they can relink them
+with the library after making changes to the library and recompiling
+it. And you must show them these terms so they know their rights.
+
+ We protect your rights with a two-step method: (1) we copyright the
+library, and (2) we offer you this license, which gives you legal
+permission to copy, distribute and/or modify the library.
+
+ To protect each distributor, we want to make it very clear that
+there is no warranty for the free library. Also, if the library is
+modified by someone else and passed on, the recipients should know
+that what they have is not the original version, so that the original
+author's reputation will not be affected by problems that might be
+introduced by others.
+
+ Finally, software patents pose a constant threat to the existence of
+any free program. We wish to make sure that a company cannot
+effectively restrict the users of a free program by obtaining a
+restrictive license from a patent holder. Therefore, we insist that
+any patent license obtained for a version of the library must be
+consistent with the full freedom of use specified in this license.
+
+ Most GNU software, including some libraries, is covered by the
+ordinary GNU General Public License. This license, the GNU Lesser
+General Public License, applies to certain designated libraries, and
+is quite different from the ordinary General Public License. We use
+this license for certain libraries in order to permit linking those
+libraries into non-free programs.
+
+ When a program is linked with a library, whether statically or using
+a shared library, the combination of the two is legally speaking a
+combined work, a derivative of the original library. The ordinary
+General Public License therefore permits such linking only if the
+entire combination fits its criteria of freedom. The Lesser General
+Public License permits more lax criteria for linking other code with
+the library.
+
+ We call this license the "Lesser" General Public License because it
+does Less to protect the user's freedom than the ordinary General
+Public License. It also provides other free software developers Less
+of an advantage over competing non-free programs. These disadvantages
+are the reason we use the ordinary General Public License for many
+libraries. However, the Lesser license provides advantages in certain
+special circumstances.
+
+ For example, on rare occasions, there may be a special need to
+encourage the widest possible use of a certain library, so that it becomes
+a de-facto standard. To achieve this, non-free programs must be
+allowed to use the library. A more frequent case is that a free
+library does the same job as widely used non-free libraries. In this
+case, there is little to gain by limiting the free library to free
+software only, so we use the Lesser General Public License.
+
+ In other cases, permission to use a particular library in non-free
+programs enables a greater number of people to use a large body of
+free software. For example, permission to use the GNU C Library in
+non-free programs enables many more people to use the whole GNU
+operating system, as well as its variant, the GNU/Linux operating
+system.
+
+ Although the Lesser General Public License is Less protective of the
+users' freedom, it does ensure that the user of a program that is
+linked with the Library has the freedom and the wherewithal to run
+that program using a modified version of the Library.
+
+ The precise terms and conditions for copying, distribution and
+modification follow. Pay close attention to the difference between a
+"work based on the library" and a "work that uses the library". The
+former contains code derived from the library, whereas the latter must
+be combined with the library in order to run.
+
+ GNU LESSER GENERAL PUBLIC LICENSE
+ TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+ 0. This License Agreement applies to any software library or other
+program which contains a notice placed by the copyright holder or
+other authorized party saying it may be distributed under the terms of
+this Lesser General Public License (also called "this License").
+Each licensee is addressed as "you".
+
+ A "library" means a collection of software functions and/or data
+prepared so as to be conveniently linked with application programs
+(which use some of those functions and data) to form executables.
+
+ The "Library", below, refers to any such software library or work
+which has been distributed under these terms. A "work based on the
+Library" means either the Library or any derivative work under
+copyright law: that is to say, a work containing the Library or a
+portion of it, either verbatim or with modifications and/or translated
+straightforwardly into another language. (Hereinafter, translation is
+included without limitation in the term "modification".)
+
+ "Source code" for a work means the preferred form of the work for
+making modifications to it. For a library, complete source code means
+all the source code for all modules it contains, plus any associated
+interface definition files, plus the scripts used to control compilation
+and installation of the library.
+
+ Activities other than copying, distribution and modification are not
+covered by this License; they are outside its scope. The act of
+running a program using the Library is not restricted, and output from
+such a program is covered only if its contents constitute a work based
+on the Library (independent of the use of the Library in a tool for
+writing it). Whether that is true depends on what the Library does
+and what the program that uses the Library does.
+
+ 1. You may copy and distribute verbatim copies of the Library's
+complete source code as you receive it, in any medium, provided that
+you conspicuously and appropriately publish on each copy an
+appropriate copyright notice and disclaimer of warranty; keep intact
+all the notices that refer to this License and to the absence of any
+warranty; and distribute a copy of this License along with the
+Library.
+
+ You may charge a fee for the physical act of transferring a copy,
+and you may at your option offer warranty protection in exchange for a
+fee.
+
+ 2. You may modify your copy or copies of the Library or any portion
+of it, thus forming a work based on the Library, and copy and
+distribute such modifications or work under the terms of Section 1
+above, provided that you also meet all of these conditions:
+
+ a) The modified work must itself be a software library.
+
+ b) You must cause the files modified to carry prominent notices
+ stating that you changed the files and the date of any change.
+
+ c) You must cause the whole of the work to be licensed at no
+ charge to all third parties under the terms of this License.
+
+ d) If a facility in the modified Library refers to a function or a
+ table of data to be supplied by an application program that uses
+ the facility, other than as an argument passed when the facility
+ is invoked, then you must make a good faith effort to ensure that,
+ in the event an application does not supply such function or
+ table, the facility still operates, and performs whatever part of
+ its purpose remains meaningful.
+
+ (For example, a function in a library to compute square roots has
+ a purpose that is entirely well-defined independent of the
+ application. Therefore, Subsection 2d requires that any
+ application-supplied function or table used by this function must
+ be optional: if the application does not supply it, the square
+ root function must still compute square roots.)
+
+These requirements apply to the modified work as a whole. If
+identifiable sections of that work are not derived from the Library,
+and can be reasonably considered independent and separate works in
+themselves, then this License, and its terms, do not apply to those
+sections when you distribute them as separate works. But when you
+distribute the same sections as part of a whole which is a work based
+on the Library, the distribution of the whole must be on the terms of
+this License, whose permissions for other licensees extend to the
+entire whole, and thus to each and every part regardless of who wrote
+it.
+
+Thus, it is not the intent of this section to claim rights or contest
+your rights to work written entirely by you; rather, the intent is to
+exercise the right to control the distribution of derivative or
+collective works based on the Library.
+
+In addition, mere aggregation of another work not based on the Library
+with the Library (or with a work based on the Library) on a volume of
+a storage or distribution medium does not bring the other work under
+the scope of this License.
+
+ 3. You may opt to apply the terms of the ordinary GNU General Public
+License instead of this License to a given copy of the Library. To do
+this, you must alter all the notices that refer to this License, so
+that they refer to the ordinary GNU General Public License, version 2,
+instead of to this License. (If a newer version than version 2 of the
+ordinary GNU General Public License has appeared, then you can specify
+that version instead if you wish.) Do not make any other change in
+these notices.
+
+ Once this change is made in a given copy, it is irreversible for
+that copy, so the ordinary GNU General Public License applies to all
+subsequent copies and derivative works made from that copy.
+
+ This option is useful when you wish to copy part of the code of
+the Library into a program that is not a library.
+
+ 4. You may copy and distribute the Library (or a portion or
+derivative of it, under Section 2) in object code or executable form
+under the terms of Sections 1 and 2 above provided that you accompany
+it with the complete corresponding machine-readable source code, which
+must be distributed under the terms of Sections 1 and 2 above on a
+medium customarily used for software interchange.
+
+ If distribution of object code is made by offering access to copy
+from a designated place, then offering equivalent access to copy the
+source code from the same place satisfies the requirement to
+distribute the source code, even though third parties are not
+compelled to copy the source along with the object code.
+
+ 5. A program that contains no derivative of any portion of the
+Library, but is designed to work with the Library by being compiled or
+linked with it, is called a "work that uses the Library". Such a
+work, in isolation, is not a derivative work of the Library, and
+therefore falls outside the scope of this License.
+
+ However, linking a "work that uses the Library" with the Library
+creates an executable that is a derivative of the Library (because it
+contains portions of the Library), rather than a "work that uses the
+library". The executable is therefore covered by this License.
+Section 6 states terms for distribution of such executables.
+
+ When a "work that uses the Library" uses material from a header file
+that is part of the Library, the object code for the work may be a
+derivative work of the Library even though the source code is not.
+Whether this is true is especially significant if the work can be
+linked without the Library, or if the work is itself a library. The
+threshold for this to be true is not precisely defined by law.
+
+ If such an object file uses only numerical parameters, data
+structure layouts and accessors, and small macros and small inline
+functions (ten lines or less in length), then the use of the object
+file is unrestricted, regardless of whether it is legally a derivative
+work. (Executables containing this object code plus portions of the
+Library will still fall under Section 6.)
+
+ Otherwise, if the work is a derivative of the Library, you may
+distribute the object code for the work under the terms of Section 6.
+Any executables containing that work also fall under Section 6,
+whether or not they are linked directly with the Library itself.
+
+ 6. As an exception to the Sections above, you may also combine or
+link a "work that uses the Library" with the Library to produce a
+work containing portions of the Library, and distribute that work
+under terms of your choice, provided that the terms permit
+modification of the work for the customer's own use and reverse
+engineering for debugging such modifications.
+
+ You must give prominent notice with each copy of the work that the
+Library is used in it and that the Library and its use are covered by
+this License. You must supply a copy of this License. If the work
+during execution displays copyright notices, you must include the
+copyright notice for the Library among them, as well as a reference
+directing the user to the copy of this License. Also, you must do one
+of these things:
+
+ a) Accompany the work with the complete corresponding
+ machine-readable source code for the Library including whatever
+ changes were used in the work (which must be distributed under
+ Sections 1 and 2 above); and, if the work is an executable linked
+ with the Library, with the complete machine-readable "work that
+ uses the Library", as object code and/or source code, so that the
+ user can modify the Library and then relink to produce a modified
+ executable containing the modified Library. (It is understood
+ that the user who changes the contents of definitions files in the
+ Library will not necessarily be able to recompile the application
+ to use the modified definitions.)
+
+ b) Use a suitable shared library mechanism for linking with the
+ Library. A suitable mechanism is one that (1) uses at run time a
+ copy of the library already present on the user's computer system,
+ rather than copying library functions into the executable, and (2)
+ will operate properly with a modified version of the library, if
+ the user installs one, as long as the modified version is
+ interface-compatible with the version that the work was made with.
+
+ c) Accompany the work with a written offer, valid for at
+ least three years, to give the same user the materials
+ specified in Subsection 6a, above, for a charge no more
+ than the cost of performing this distribution.
+
+ d) If distribution of the work is made by offering access to copy
+ from a designated place, offer equivalent access to copy the above
+ specified materials from the same place.
+
+ e) Verify that the user has already received a copy of these
+ materials or that you have already sent this user a copy.
+
+ For an executable, the required form of the "work that uses the
+Library" must include any data and utility programs needed for
+reproducing the executable from it. However, as a special exception,
+the materials to be distributed need not include anything that is
+normally distributed (in either source or binary form) with the major
+components (compiler, kernel, and so on) of the operating system on
+which the executable runs, unless that component itself accompanies
+the executable.
+
+ It may happen that this requirement contradicts the license
+restrictions of other proprietary libraries that do not normally
+accompany the operating system. Such a contradiction means you cannot
+use both them and the Library together in an executable that you
+distribute.
+
+ 7. You may place library facilities that are a work based on the
+Library side-by-side in a single library together with other library
+facilities not covered by this License, and distribute such a combined
+library, provided that the separate distribution of the work based on
+the Library and of the other library facilities is otherwise
+permitted, and provided that you do these two things:
+
+ a) Accompany the combined library with a copy of the same work
+ based on the Library, uncombined with any other library
+ facilities. This must be distributed under the terms of the
+ Sections above.
+
+ b) Give prominent notice with the combined library of the fact
+ that part of it is a work based on the Library, and explaining
+ where to find the accompanying uncombined form of the same work.
+
+ 8. You may not copy, modify, sublicense, link with, or distribute
+the Library except as expressly provided under this License. Any
+attempt otherwise to copy, modify, sublicense, link with, or
+distribute the Library is void, and will automatically terminate your
+rights under this License. However, parties who have received copies,
+or rights, from you under this License will not have their licenses
+terminated so long as such parties remain in full compliance.
+
+ 9. You are not required to accept this License, since you have not
+signed it. However, nothing else grants you permission to modify or
+distribute the Library or its derivative works. These actions are
+prohibited by law if you do not accept this License. Therefore, by
+modifying or distributing the Library (or any work based on the
+Library), you indicate your acceptance of this License to do so, and
+all its terms and conditions for copying, distributing or modifying
+the Library or works based on it.
+
+ 10. Each time you redistribute the Library (or any work based on the
+Library), the recipient automatically receives a license from the
+original licensor to copy, distribute, link with or modify the Library
+subject to these terms and conditions. You may not impose any further
+restrictions on the recipients' exercise of the rights granted herein.
+You are not responsible for enforcing compliance by third parties with
+this License.
+
+ 11. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues),
+conditions are imposed on you (whether by court order, agreement or
+otherwise) that contradict the conditions of this License, they do not
+excuse you from the conditions of this License. If you cannot
+distribute so as to satisfy simultaneously your obligations under this
+License and any other pertinent obligations, then as a consequence you
+may not distribute the Library at all. For example, if a patent
+license would not permit royalty-free redistribution of the Library by
+all those who receive copies directly or indirectly through you, then
+the only way you could satisfy both it and this License would be to
+refrain entirely from distribution of the Library.
+
+If any portion of this section is held invalid or unenforceable under any
+particular circumstance, the balance of the section is intended to apply,
+and the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any
+patents or other property right claims or to contest validity of any
+such claims; this section has the sole purpose of protecting the
+integrity of the free software distribution system which is
+implemented by public license practices. Many people have made
+generous contributions to the wide range of software distributed
+through that system in reliance on consistent application of that
+system; it is up to the author/donor to decide if he or she is willing
+to distribute software through any other system and a licensee cannot
+impose that choice.
+
+This section is intended to make thoroughly clear what is believed to
+be a consequence of the rest of this License.
+
+ 12. If the distribution and/or use of the Library is restricted in
+certain countries either by patents or by copyrighted interfaces, the
+original copyright holder who places the Library under this License may add
+an explicit geographical distribution limitation excluding those countries,
+so that distribution is permitted only in or among countries not thus
+excluded. In such case, this License incorporates the limitation as if
+written in the body of this License.
+
+ 13. The Free Software Foundation may publish revised and/or new
+versions of the Lesser General Public License from time to time.
+Such new versions will be similar in spirit to the present version,
+but may differ in detail to address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Library
+specifies a version number of this License which applies to it and
+"any later version", you have the option of following the terms and
+conditions either of that version or of any later version published by
+the Free Software Foundation. If the Library does not specify a
+license version number, you may choose any version ever published by
+the Free Software Foundation.
+
+ 14. If you wish to incorporate parts of the Library into other free
+programs whose distribution conditions are incompatible with these,
+write to the author to ask for permission. For software which is
+copyrighted by the Free Software Foundation, write to the Free
+Software Foundation; we sometimes make exceptions for this. Our
+decision will be guided by the two goals of preserving the free status
+of all derivatives of our free software and of promoting the sharing
+and reuse of software generally.
+
+ NO WARRANTY
+
+ 15. BECAUSE THE LIBRARY IS LICENSED FREE OF CHARGE, THERE IS NO
+WARRANTY FOR THE LIBRARY, TO THE EXTENT PERMITTED BY APPLICABLE LAW.
+EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR
+OTHER PARTIES PROVIDE THE LIBRARY "AS IS" WITHOUT WARRANTY OF ANY
+KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE
+LIBRARY IS WITH YOU. SHOULD THE LIBRARY PROVE DEFECTIVE, YOU ASSUME
+THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+ 16. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN
+WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY
+AND/OR REDISTRIBUTE THE LIBRARY AS PERMITTED ABOVE, BE LIABLE TO YOU
+FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR
+CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE
+LIBRARY (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING
+RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A
+FAILURE OF THE LIBRARY TO OPERATE WITH ANY OTHER SOFTWARE), EVEN IF
+SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH
+DAMAGES.
+
+ END OF TERMS AND CONDITIONS
diff --git a/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.105.Final.LICENSE.txt
similarity index 99%
rename from zookeeper-server/src/main/resources/lib/netty-buffer-4.1.73.Final.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/netty-buffer-4.1.105.Final.LICENSE.txt
index 6279e5206de..2e16f8158c9 100644
--- a/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.73.Final.LICENSE.txt
+++ b/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.105.Final.LICENSE.txt
@@ -1,7 +1,7 @@
Apache License
Version 2.0, January 2004
- http://www.apache.org/licenses/
+ https://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
@@ -193,7 +193,7 @@
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
+ https://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,
diff --git a/zookeeper-server/src/main/resources/lib/netty-codec-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-codec-4.1.105.Final.LICENSE.txt
similarity index 99%
rename from zookeeper-server/src/main/resources/lib/netty-codec-4.1.73.Final.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/netty-codec-4.1.105.Final.LICENSE.txt
index 6279e5206de..2e16f8158c9 100644
--- a/zookeeper-server/src/main/resources/lib/netty-codec-4.1.73.Final.LICENSE.txt
+++ b/zookeeper-server/src/main/resources/lib/netty-codec-4.1.105.Final.LICENSE.txt
@@ -1,7 +1,7 @@
Apache License
Version 2.0, January 2004
- http://www.apache.org/licenses/
+ https://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
@@ -193,7 +193,7 @@
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
+ https://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,
diff --git a/zookeeper-server/src/main/resources/lib/netty-common-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-common-4.1.105.Final.LICENSE.txt
similarity index 99%
rename from zookeeper-server/src/main/resources/lib/netty-common-4.1.73.Final.LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/netty-common-4.1.105.Final.LICENSE.txt
index 6279e5206de..2e16f8158c9 100644
--- a/zookeeper-server/src/main/resources/lib/netty-common-4.1.73.Final.LICENSE.txt
+++ b/zookeeper-server/src/main/resources/lib/netty-common-4.1.105.Final.LICENSE.txt
@@ -1,7 +1,7 @@
Apache License
Version 2.0, January 2004
- http://www.apache.org/licenses/
+ https://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
@@ -193,7 +193,7 @@
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
+ https://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,
diff --git a/zookeeper-server/src/main/resources/lib/netty-handler-4.1.105.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-handler-4.1.105.Final.LICENSE.txt
new file mode 100644
index 00000000000..2e16f8158c9
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/netty-handler-4.1.105.Final.LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ https://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 1999-2005 The Apache Software Foundation
+
+ 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
+
+ https://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/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.105.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.105.Final.LICENSE.txt
new file mode 100644
index 00000000000..2e16f8158c9
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.105.Final.LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ https://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 1999-2005 The Apache Software Foundation
+
+ 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
+
+ https://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/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.73.Final.LICENSE.txt
deleted file mode 100644
index 6279e5206de..00000000000
--- a/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.73.Final.LICENSE.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 1999-2005 The Apache Software Foundation
-
- 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/zookeeper-server/src/main/resources/lib/netty-tcnative-classes-2.0.48.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-tcnative-classes-2.0.48.Final.LICENSE.txt
deleted file mode 100644
index 6279e5206de..00000000000
--- a/zookeeper-server/src/main/resources/lib/netty-tcnative-classes-2.0.48.Final.LICENSE.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 1999-2005 The Apache Software Foundation
-
- 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/zookeeper-server/src/main/resources/lib/netty-transport-4.1.105.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-4.1.105.Final.LICENSE.txt
new file mode 100644
index 00000000000..2e16f8158c9
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/netty-transport-4.1.105.Final.LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ https://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 1999-2005 The Apache Software Foundation
+
+ 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
+
+ https://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/zookeeper-server/src/main/resources/lib/netty-transport-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-4.1.73.Final.LICENSE.txt
deleted file mode 100644
index 6279e5206de..00000000000
--- a/zookeeper-server/src/main/resources/lib/netty-transport-4.1.73.Final.LICENSE.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 1999-2005 The Apache Software Foundation
-
- 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/zookeeper-server/src/main/resources/lib/netty-transport-classes-epoll-4.1.105.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-classes-epoll-4.1.105.Final.LICENSE.txt
new file mode 100644
index 00000000000..2e16f8158c9
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/netty-transport-classes-epoll-4.1.105.Final.LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ https://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 1999-2005 The Apache Software Foundation
+
+ 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
+
+ https://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/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.105.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.105.Final.LICENSE.txt
new file mode 100644
index 00000000000..2e16f8158c9
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.105.Final.LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ https://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 1999-2005 The Apache Software Foundation
+
+ 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
+
+ https://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/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.73.Final.LICENSE.txt
deleted file mode 100644
index 6279e5206de..00000000000
--- a/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.73.Final.LICENSE.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 1999-2005 The Apache Software Foundation
-
- 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/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.105.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.105.Final.LICENSE.txt
new file mode 100644
index 00000000000..2e16f8158c9
--- /dev/null
+++ b/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.105.Final.LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ https://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 1999-2005 The Apache Software Foundation
+
+ 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
+
+ https://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/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.73.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.73.Final.LICENSE.txt
deleted file mode 100644
index 6279e5206de..00000000000
--- a/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.73.Final.LICENSE.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 1999-2005 The Apache Software Foundation
-
- 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/zookeeper-server/src/main/resources/lib/snappy-java-1.1.7.7.jar_LICENSE.txt b/zookeeper-server/src/main/resources/lib/snappy-java-1.1.10.5.jar_LICENSE.txt
similarity index 100%
rename from zookeeper-server/src/main/resources/lib/snappy-java-1.1.7.7.jar_LICENSE.txt
rename to zookeeper-server/src/main/resources/lib/snappy-java-1.1.10.5.jar_LICENSE.txt
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/KerberosTicketRenewalTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/KerberosTicketRenewalTest.java
new file mode 100644
index 00000000000..84b214049f0
--- /dev/null
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/KerberosTicketRenewalTest.java
@@ -0,0 +1,278 @@
+/*
+ * 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.zookeeper;
+
+import static org.apache.zookeeper.server.quorum.auth.MiniKdc.MAX_TICKET_LIFETIME;
+import static org.apache.zookeeper.server.quorum.auth.MiniKdc.MIN_TICKET_LIFETIME;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTimeout;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.security.Principal;
+import java.time.Duration;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Supplier;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.zookeeper.common.ZKConfig;
+import org.apache.zookeeper.server.quorum.auth.KerberosTestUtils;
+import org.apache.zookeeper.server.quorum.auth.MiniKdc;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test class is mainly testing the TGT renewal logic implemented
+ * in the org.apache.zookeeper.Login class.
+ */
+public class KerberosTicketRenewalTest {
+
+
+ private static final Logger LOG = LoggerFactory.getLogger(KerberosTicketRenewalTest.class);
+ private static final String JAAS_CONFIG_SECTION = "ClientUsingKerberos";
+ private static final String TICKET_LIFETIME = "5000";
+ private static File testTempDir;
+ private static MiniKdc kdc;
+ private static File kdcWorkDir;
+ private static String PRINCIPAL = KerberosTestUtils.getClientPrincipal();
+
+ TestableKerberosLogin login;
+
+ @BeforeAll
+ public static void setupClass() throws Exception {
+ // by default, we should wait at least 1 minute between subsequent TGT renewals.
+ // changing it to 500ms.
+ System.setProperty(Login.MIN_TIME_BEFORE_RELOGIN_CONFIG_KEY, "500");
+
+ testTempDir = ClientBase.createTmpDir();
+ startMiniKdcAndAddPrincipal();
+
+ String keytabFilePath = FilenameUtils.normalize(KerberosTestUtils.getKeytabFile(), true);
+
+ // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM,
+ // making sure that we use the latest config even if other tests already have been executed
+ // and initialized the kerberos client configs before)
+ String jaasEntries = ""
+ + "ClientUsingKerberos {\n"
+ + " com.sun.security.auth.module.Krb5LoginModule required\n"
+ + " storeKey=\"false\"\n"
+ + " useTicketCache=\"false\"\n"
+ + " useKeyTab=\"true\"\n"
+ + " doNotPrompt=\"true\"\n"
+ + " debug=\"true\"\n"
+ + " refreshKrb5Config=\"true\"\n"
+ + " keyTab=\"" + keytabFilePath + "\"\n"
+ + " principal=\"" + PRINCIPAL + "\";\n"
+ + "};\n";
+ setupJaasConfig(jaasEntries);
+ }
+
+ @AfterAll
+ public static void tearDownClass() {
+ System.clearProperty(Login.MIN_TIME_BEFORE_RELOGIN_CONFIG_KEY);
+ System.clearProperty("java.security.auth.login.config");
+ stopMiniKdc();
+ if (testTempDir != null) {
+ // the testTempDir contains the jaas config file and also the
+ // working folder of the currently running KDC server
+ FileUtils.deleteQuietly(testTempDir);
+ }
+ }
+
+ @AfterEach
+ public void tearDownTest() throws Exception {
+ if (login != null) {
+ login.shutdown();
+ login.logout();
+ }
+ }
+
+
+ /**
+ * We extend the regular Login class to be able to properly control the
+ * "sleeping" between the retry attempts of ticket refresh actions.
+ */
+ private static class TestableKerberosLogin extends Login {
+
+ private AtomicBoolean refreshFailed = new AtomicBoolean(false);
+ private CountDownLatch continueRefreshThread = new CountDownLatch(1);
+
+ public TestableKerberosLogin() throws LoginException {
+ super(JAAS_CONFIG_SECTION, (callbacks) -> {}, new ZKConfig());
+ }
+
+ @Override
+ protected void sleepBeforeRetryFailedRefresh() throws InterruptedException {
+ LOG.info("sleep started due to failed refresh");
+ refreshFailed.set(true);
+ continueRefreshThread.await(20, TimeUnit.SECONDS);
+ LOG.info("sleep due to failed refresh finished");
+ }
+
+ public void assertRefreshFailsEventually(Duration timeout) {
+ assertEventually(timeout, () -> refreshFailed.get());
+ }
+
+ public void continueWithRetryAfterFailedRefresh() {
+ LOG.info("continue refresh thread");
+ continueRefreshThread.countDown();
+ }
+ }
+
+
+ @Test
+ public void shouldLoginUsingKerberos() throws Exception {
+ login = new TestableKerberosLogin();
+ login.startThreadIfNeeded();
+
+ assertPrincipalLoggedIn();
+ }
+
+
+ @Test
+ public void shouldRenewTicketUsingKerberos() throws Exception {
+ login = new TestableKerberosLogin();
+ login.startThreadIfNeeded();
+
+ long initialLoginTime = login.getLastLogin();
+
+ // ticket lifetime is 5sec, so we will trigger ticket renewal in each ~2-3 sec
+ assertTicketRefreshHappenedUntil(Duration.ofSeconds(15));
+
+ assertPrincipalLoggedIn();
+ assertTrue(initialLoginTime < login.getLastLogin());
+ }
+
+
+ @Test
+ public void shouldRecoverIfKerberosNotAvailableForSomeTime() throws Exception {
+ login = new TestableKerberosLogin();
+ login.startThreadIfNeeded();
+
+ assertTicketRefreshHappenedUntil(Duration.ofSeconds(15));
+
+ stopMiniKdc();
+
+ // ticket lifetime is 5sec, so we will trigger ticket renewal in each ~2-3 sec
+ // the very next ticket renewal should fail (as KDC is offline)
+ login.assertRefreshFailsEventually(Duration.ofSeconds(15));
+
+ // now the ticket thread is "sleeping", it will retry the refresh later
+
+ // we restart KDC, then terminate the "sleeping" and expecting
+ // that the next retry should succeed
+ startMiniKdcAndAddPrincipal();
+ login.continueWithRetryAfterFailedRefresh();
+ assertTicketRefreshHappenedUntil(Duration.ofSeconds(15));
+
+ assertPrincipalLoggedIn();
+ }
+
+
+ private void assertPrincipalLoggedIn() {
+ assertEquals(PRINCIPAL, login.getUserName());
+ assertNotNull(login.getSubject());
+ assertEquals(1, login.getSubject().getPrincipals().size());
+ Principal actualPrincipal = login.getSubject().getPrincipals().iterator().next();
+ assertEquals(PRINCIPAL, actualPrincipal.getName());
+ }
+
+ private void assertTicketRefreshHappenedUntil(Duration timeout) {
+ long lastLoginTime = login.getLastLogin();
+ assertEventually(timeout, () -> login.getLastLogin() != lastLoginTime
+ && login.getSubject() != null && !login.getSubject().getPrincipals().isEmpty());
+ }
+
+ private static void assertEventually(Duration timeout, Supplier test) {
+ assertTimeout(timeout, () -> {
+ while (true) {
+ if (test.get()) {
+ return;
+ }
+ Thread.sleep(100);
+ }
+ });
+ }
+
+ public static void startMiniKdcAndAddPrincipal() throws Exception {
+ kdcWorkDir = createTmpDirInside(testTempDir);
+
+ Properties conf = MiniKdc.createConf();
+ conf.setProperty(MAX_TICKET_LIFETIME, TICKET_LIFETIME);
+ conf.setProperty(MIN_TICKET_LIFETIME, TICKET_LIFETIME);
+
+ kdc = new MiniKdc(conf, kdcWorkDir);
+ kdc.start();
+
+ String principalName = PRINCIPAL.substring(0, PRINCIPAL.lastIndexOf("@"));
+ kdc.createPrincipal(new File(KerberosTestUtils.getKeytabFile()), principalName);
+ }
+
+ private static void stopMiniKdc() {
+ if (kdc != null) {
+ kdc.stop();
+ kdc = null;
+ }
+ if (kdcWorkDir != null) {
+ FileUtils.deleteQuietly(kdcWorkDir);
+ kdcWorkDir = null;
+ }
+ }
+
+ private static File createTmpDirInside(File parentDir) throws IOException {
+ File tmpFile = File.createTempFile("test", ".junit", parentDir);
+ // don't delete tmpFile - this ensures we don't attempt to create
+ // a tmpDir with a duplicate name
+ File tmpDir = new File(tmpFile + ".dir");
+ // never true if tmpfile does it's job
+ assertFalse(tmpDir.exists());
+ assertTrue(tmpDir.mkdirs());
+ return tmpDir;
+ }
+
+ private static void setupJaasConfig(String jaasEntries) {
+ try {
+ File saslConfFile = new File(testTempDir, "jaas.conf");
+ FileWriter fwriter = new FileWriter(saslConfFile);
+ fwriter.write(jaasEntries);
+ fwriter.close();
+ System.setProperty("java.security.auth.login.config", saslConfFile.getAbsolutePath());
+ } catch (IOException ioe) {
+ LOG.error("Failed to initialize JAAS conf file", ioe);
+ }
+
+ // refresh the SASL configuration in this JVM (making sure that we use the latest config
+ // even if other tests already have been executed and initialized the SASL configs before)
+ Configuration.getConfiguration().refresh();
+ }
+
+}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/MultiOperationRecordTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/MultiOperationRecordTest.java
index 54bf12f4e9c..df8d8b91642 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/MultiOperationRecordTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/MultiOperationRecordTest.java
@@ -33,7 +33,7 @@ public class MultiOperationRecordTest extends ZKTestCase {
public void testRoundTrip() throws IOException {
MultiOperationRecord request = new MultiOperationRecord();
request.add(Op.check("check", 1));
- request.add(Op.create("create", "create data".getBytes(), ZooDefs.Ids.CREATOR_ALL_ACL, ZooDefs.Perms.ALL));
+ request.add(Op.create("create", "create data".getBytes(), ZooDefs.Ids.CREATOR_ALL_ACL, CreateMode.EPHEMERAL.toFlag()));
request.add(Op.delete("delete", 17));
request.add(Op.setData("setData", "set data".getBytes(), 19));
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java
index 005cf4bac30..6bb768e8fa1 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java
@@ -20,6 +20,7 @@
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@@ -239,12 +240,9 @@ public void testThreadsShutdownOnAuthFailed() throws Exception {
assertNotNull(zooKeeperSaslClient);
sendThread.join(CONNECTION_TIMEOUT);
eventThread.join(CONNECTION_TIMEOUT);
- Field loginField = zooKeeperSaslClient.getClass().getDeclaredField("login");
- loginField.setAccessible(true);
- Login login = (Login) loginField.get(zooKeeperSaslClient);
// If login is null, this means ZooKeeperSaslClient#shutdown method has been called which in turns
// means that Login#shutdown has been called.
- assertNull(login);
+ assertNull(sendThread.getLogin());
assertFalse(sendThread.isAlive(), "SendThread did not shutdown after authFail");
assertFalse(eventThread.isAlive(), "EventThread did not shutdown after authFail");
} finally {
@@ -253,4 +251,37 @@ public void testThreadsShutdownOnAuthFailed() throws Exception {
}
}
}
-}
\ No newline at end of file
+
+ @Test
+ public void testDisconnectNotCreatingLoginThread() throws Exception {
+ MyWatcher watcher = new MyWatcher();
+ ZooKeeper zk = null;
+ try {
+ zk = new ZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher);
+ watcher.waitForConnected(CONNECTION_TIMEOUT);
+ zk.getData("/", false, null);
+
+ Field cnxnField = zk.getClass().getDeclaredField("cnxn");
+ cnxnField.setAccessible(true);
+ ClientCnxn clientCnxn = (ClientCnxn) cnxnField.get(zk);
+ Field sendThreadField = clientCnxn.getClass().getDeclaredField("sendThread");
+ sendThreadField.setAccessible(true);
+ SendThread sendThread = (SendThread) sendThreadField.get(clientCnxn);
+
+ Login l1 = sendThread.getLogin();
+ assertNotNull(l1);
+
+ stopServer();
+ watcher.waitForDisconnected(CONNECTION_TIMEOUT);
+ startServer();
+ watcher.waitForConnected(CONNECTION_TIMEOUT);
+ zk.getData("/", false, null);
+
+ assertSame("Login thread should not been recreated on disconnect", l1, sendThread.getLogin());
+ } finally {
+ if (zk != null) {
+ zk.close();
+ }
+ }
+ }
+}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java
index 21343c161ca..eb8fcbf934b 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java
@@ -21,7 +21,7 @@
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import java.io.File;
-import java.time.LocalDateTime;
+import java.time.Instant;
import org.apache.zookeeper.util.ServiceUtils;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
@@ -100,8 +100,8 @@ public interface WaitForCondition {
* @throws InterruptedException
*/
public void waitFor(String msg, WaitForCondition condition, int timeout) throws InterruptedException {
- final LocalDateTime deadline = LocalDateTime.now().plusSeconds(timeout);
- while (LocalDateTime.now().isBefore(deadline)) {
+ final Instant deadline = Instant.now().plusSeconds(timeout);
+ while (Instant.now().isBefore(deadline)) {
if (condition.evaluate()) {
return;
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ZKUtilTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ZKUtilTest.java
index 51df907023a..a73ff2fa7e0 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/ZKUtilTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ZKUtilTest.java
@@ -25,10 +25,12 @@
import static org.junit.jupiter.api.Assertions.assertNull;
import java.io.File;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.UUID;
+import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.test.ClientBase;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
@@ -119,4 +121,52 @@ public void testListNoneRootPathSuccess() throws IOException, InterruptedExcepti
assertEquals(1, bList.size());
assertIterableEquals(Collections.singletonList("/a/b"), bList);
}
+
+ @Test
+ public void testDeleteRecursiveInAsyncMode() throws Exception {
+ int batchSize = 10;
+ testDeleteRecursiveInSyncAsyncMode(batchSize);
+ }
+
+ @Test
+ public void testDeleteRecursiveInSyncMode() throws Exception {
+ int batchSize = 0;
+ testDeleteRecursiveInSyncAsyncMode(batchSize);
+ }
+
+ // batchSize>0 is async mode otherwise it is sync mode
+ private void testDeleteRecursiveInSyncAsyncMode(int batchSize)
+ throws IOException, InterruptedException, KeeperException {
+ TestableZooKeeper zk = createClient();
+ String parentPath = "/a";
+ zk.create(parentPath, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ int numberOfNodes = 50;
+ List ops = new ArrayList<>();
+ for (int i = 0; i < numberOfNodes; i++) {
+ ops.add(Op.create(parentPath + "/a" + i, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT));
+ }
+ zk.multi(ops);
+ ops.clear();
+
+ // check nodes create successfully
+ List children = zk.getChildren(parentPath, false);
+ assertEquals(numberOfNodes, children.size());
+
+ // create one more level of z nodes
+ String subNode = "/a/a0";
+ for (int i = 0; i < numberOfNodes; i++) {
+ ops.add(Op.create(subNode + "/b" + i, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT));
+ }
+ zk.multi(ops);
+
+ // check sub nodes created successfully
+ children = zk.getChildren(subNode, false);
+ assertEquals(numberOfNodes, children.size());
+
+ ZKUtil.deleteRecursive(zk, parentPath, batchSize);
+ Stat exists = zk.exists(parentPath, false);
+ assertNull(exists, "ZKUtil.deleteRecursive() could not delete all the z nodes");
+ }
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/NetUtilsTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/NetUtilsTest.java
index a7006bf7053..fbe49221470 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/NetUtilsTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/NetUtilsTest.java
@@ -18,7 +18,6 @@
package org.apache.zookeeper.common;
-import static org.hamcrest.CoreMatchers.anyOf;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -40,7 +39,7 @@ public class NetUtilsTest extends ZKTestCase {
@Test
public void testFormatInetAddrGoodIpv4() {
InetSocketAddress isa = new InetSocketAddress(v4addr, port);
- assertEquals("127.0.0.1:1234", NetUtils.formatInetAddr(isa));
+ assertEquals(v4local, NetUtils.formatInetAddr(isa));
}
@Test
@@ -60,8 +59,7 @@ public void testFormatInetAddrGoodIpv6Ext() {
@Test
public void testFormatInetAddrGoodHostname() {
InetSocketAddress isa = new InetSocketAddress("localhost", 1234);
-
- assertThat(NetUtils.formatInetAddr(isa), anyOf(equalTo(v4local), equalTo(v6local)));
+ assertThat(NetUtils.formatInetAddr(isa), equalTo("localhost:1234"));
}
@Test
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java
index c6812ae0414..ddffd426dd8 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java
@@ -360,7 +360,8 @@ public void testLoadPEMTrustStore(
false,
false,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -380,7 +381,8 @@ public void testLoadPEMTrustStoreNullPassword(
false,
false,
true,
- true);
+ true,
+ false);
}
@@ -398,7 +400,8 @@ public void testLoadPEMTrustStoreAutodetectStoreFileType(
false,
false,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -472,7 +475,8 @@ public void testLoadJKSTrustStore(
true,
true,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -492,7 +496,8 @@ public void testLoadJKSTrustStoreNullPassword(
false,
false,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -509,7 +514,8 @@ public void testLoadJKSTrustStoreAutodetectStoreFileType(
true,
true,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -527,7 +533,8 @@ public void testLoadJKSTrustStoreWithWrongPassword(
true,
true,
true,
- true);
+ true,
+ false);
});
}
@@ -601,7 +608,8 @@ public void testLoadPKCS12TrustStore(
true,
true,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -621,7 +629,8 @@ public void testLoadPKCS12TrustStoreNullPassword(
false,
false,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -638,7 +647,8 @@ public void testLoadPKCS12TrustStoreAutodetectStoreFileType(
true,
true,
true,
- true);
+ true,
+ false);
}
@ParameterizedTest
@@ -656,7 +666,8 @@ public void testLoadPKCS12TrustStoreWithWrongPassword(
true,
true,
true,
- true);
+ true,
+ false);
});
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java
index c549a9ce9d6..7b4e8783ee3 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java
@@ -18,6 +18,8 @@
package org.apache.zookeeper.common;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@@ -30,10 +32,16 @@
import java.security.Security;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Calendar;
import java.util.Date;
+import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Random;
+import java.util.concurrent.CopyOnWriteArrayList;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.SSLSession;
import javax.net.ssl.X509ExtendedTrustManager;
import org.apache.zookeeper.ZKTestCase;
import org.bouncycastle.asn1.x500.X500NameBuilder;
@@ -49,25 +57,46 @@
import org.bouncycastle.jce.provider.BouncyCastleProvider;
import org.bouncycastle.operator.ContentSigner;
import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+import org.burningwave.tools.net.DefaultHostResolver;
+import org.burningwave.tools.net.HostResolutionRequestInterceptor;
+import org.burningwave.tools.net.MappedHostResolver;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
// We can only test calls to ZKTrustManager using Sockets (not SSLEngines). This can be fine since the logic is the same.
public class ZKTrustManagerTest extends ZKTestCase {
+ protected static final Logger LOG = LoggerFactory.getLogger(ZKTrustManagerTest.class);
+
private static KeyPair keyPair;
private X509ExtendedTrustManager mockX509ExtendedTrustManager;
private static final String IP_ADDRESS = "127.0.0.1";
private static final String HOSTNAME = "localhost";
-
- private InetAddress mockInetAddress;
private Socket mockSocket;
+ @BeforeAll
+ public static void setupDNSMocks() {
+ Map hostAliases = new LinkedHashMap<>();
+ hostAliases.put(HOSTNAME, IP_ADDRESS);
+
+ HostResolutionRequestInterceptor.INSTANCE.install(
+ new MappedHostResolver(hostAliases),
+ DefaultHostResolver.INSTANCE
+ );
+ }
+
+ @AfterAll
+ public static void clearDNSMocks() {
+ HostResolutionRequestInterceptor.INSTANCE.uninstall();
+ }
+
@BeforeAll
public static void createKeyPair() throws Exception {
Security.addProvider(new BouncyCastleProvider());
@@ -85,20 +114,7 @@ public static void removeBouncyCastleProvider() throws Exception {
public void setup() throws Exception {
mockX509ExtendedTrustManager = mock(X509ExtendedTrustManager.class);
- mockInetAddress = mock(InetAddress.class);
- when(mockInetAddress.getHostAddress()).thenAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
- return IP_ADDRESS;
- }
- });
-
- when(mockInetAddress.getHostName()).thenAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
- return HOSTNAME;
- }
- });
+ InetAddress mockInetAddress = InetAddress.getByName(HOSTNAME);
mockSocket = mock(Socket.class);
when(mockSocket.getInetAddress()).thenAnswer(new Answer() {
@@ -143,106 +159,140 @@ private X509Certificate[] createSelfSignedCertifcateChain(String ipAddress, Stri
@Test
public void testServerHostnameVerificationWithHostnameVerificationDisabled() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, false);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, false,
+ hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, HOSTNAME);
zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
-
- verify(mockInetAddress, times(0)).getHostAddress();
- verify(mockInetAddress, times(0)).getHostName();
+ verify(mockSocket, times(0)).getInetAddress();
+ assertTrue(hostnameVerifier.hosts.isEmpty());
verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
}
@Test
public void testServerHostnameVerificationWithHostnameVerificationDisabledAndClientHostnameVerificationEnabled() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, true);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, true,
+ hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, HOSTNAME);
zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
+ verify(mockSocket, times(0)).getInetAddress();
- verify(mockInetAddress, times(0)).getHostAddress();
- verify(mockInetAddress, times(0)).getHostName();
+ assertTrue(hostnameVerifier.hosts.isEmpty());
verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
}
@Test
public void testServerHostnameVerificationWithIPAddress() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false,
+ hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, null);
zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
+ verify(mockSocket, times(1)).getInetAddress();
- verify(mockInetAddress, times(1)).getHostAddress();
- verify(mockInetAddress, times(0)).getHostName();
+ assertEquals(Arrays.asList(IP_ADDRESS), hostnameVerifier.hosts);
verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
}
@Test
public void testServerHostnameVerificationWithHostname() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false,
+ hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
+ verify(mockSocket, times(1)).getInetAddress();
- verify(mockInetAddress, times(1)).getHostAddress();
- verify(mockInetAddress, times(1)).getHostName();
+ assertEquals(Arrays.asList(IP_ADDRESS, HOSTNAME), hostnameVerifier.hosts);
verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
}
@Test
public void testClientHostnameVerificationWithHostnameVerificationDisabled() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, true);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, true,
+ hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+ verify(mockSocket, times(1)).getInetAddress();
- verify(mockInetAddress, times(1)).getHostAddress();
- verify(mockInetAddress, times(1)).getHostName();
+ assertEquals(Arrays.asList(IP_ADDRESS, HOSTNAME), hostnameVerifier.hosts);
verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
}
@Test
public void testClientHostnameVerificationWithClientHostnameVerificationDisabled() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true,
+ false, hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+ verify(mockSocket, times(0)).getInetAddress();
- verify(mockInetAddress, times(0)).getHostAddress();
- verify(mockInetAddress, times(0)).getHostName();
+ assertTrue(hostnameVerifier.hosts.isEmpty());
verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
}
@Test
public void testClientHostnameVerificationWithIPAddress() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, true);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, true,
+ hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, null);
zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+ verify(mockSocket, times(1)).getInetAddress();
- verify(mockInetAddress, times(1)).getHostAddress();
- verify(mockInetAddress, times(0)).getHostName();
+ assertEquals(Arrays.asList(IP_ADDRESS), hostnameVerifier.hosts);
verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
}
@Test
public void testClientHostnameVerificationWithHostname() throws Exception {
- ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, true);
+ VerifiableHostnameVerifier hostnameVerifier = new VerifiableHostnameVerifier();
+ ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, true,
+ hostnameVerifier);
X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+ verify(mockSocket, times(1)).getInetAddress();
- verify(mockInetAddress, times(1)).getHostAddress();
- verify(mockInetAddress, times(1)).getHostName();
+ assertEquals(Arrays.asList(IP_ADDRESS, HOSTNAME), hostnameVerifier.hosts);
verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
}
+
+ static class VerifiableHostnameVerifier extends ZKHostnameVerifier {
+
+ List hosts = new CopyOnWriteArrayList<>();
+
+ @Override
+ public boolean verify(String host, SSLSession session) {
+ throw new IllegalArgumentException("not expected to be called by these tests");
+ }
+
+ @Override
+ void verify(String host, X509Certificate cert) throws SSLException {
+ LOG.info("verifyWithX509Certificate {} {}", host, cert);
+ hosts.add(host);
+ super.verify(host, cert);
+ }
+ }
+
+
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ConnectionMetricsTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ConnectionMetricsTest.java
index 45d5c67ecf9..ac734d2969f 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ConnectionMetricsTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ConnectionMetricsTest.java
@@ -57,8 +57,8 @@ public void testRevalidateCount() throws Exception {
util.enableLocalSession(false);
util.startAll();
- int follower1 = (int) util.getFollowerQuorumPeers().get(0).getId();
- int follower2 = (int) util.getFollowerQuorumPeers().get(1).getId();
+ int follower1 = (int) util.getFollowerQuorumPeers().get(0).getMyId();
+ int follower2 = (int) util.getFollowerQuorumPeers().get(1).getMyId();
LOG.info("connecting to server: {}", follower1);
ClientBase.CountdownWatcher watcher = new ClientBase.CountdownWatcher();
// create a connection to follower
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateTTLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateTTLTest.java
index 2cf5db4a592..ba13a0be573 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateTTLTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateTTLTest.java
@@ -30,6 +30,7 @@
import java.util.concurrent.atomic.AtomicLong;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.CreateOptions;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.Op;
@@ -182,29 +183,46 @@ public void testModifying() throws KeeperException, InterruptedException {
@Test
public void testMulti() throws KeeperException, InterruptedException {
- Op createTtl = Op.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_WITH_TTL, 100);
- Op createTtlSequential = Op.create("/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL_WITH_TTL, 200);
+ CreateOptions options = CreateOptions
+ .newBuilder(ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_WITH_TTL)
+ .withTtl(100)
+ .build();
+ CreateOptions sequentialOptions = CreateOptions
+ .newBuilder(ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL_WITH_TTL)
+ .withTtl(200)
+ .build();
+ Op createTtl = Op.create("/a", new byte[0], options.getAcl(), options.getCreateMode(), options.getTtl());
+ Op createTtl2 = Op.create("/a2", new byte[0], options);
+ Op createTtlSequential = Op.create("/b", new byte[0], sequentialOptions.getAcl(), sequentialOptions.getCreateMode(), sequentialOptions.getTtl());
+ Op createTtlSequential2 = Op.create("/b2", new byte[0], sequentialOptions);
Op createNonTtl = Op.create("/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- List results = zk.multi(Arrays.asList(createTtl, createTtlSequential, createNonTtl));
- String sequentialPath = ((OpResult.CreateResult) results.get(1)).getPath();
+ List results = zk.multi(Arrays.asList(createTtl, createTtl2, createTtlSequential, createTtlSequential2, createNonTtl));
+ String sequentialPath = ((OpResult.CreateResult) results.get(2)).getPath();
+ String sequentialPath2 = ((OpResult.CreateResult) results.get(3)).getPath();
final AtomicLong fakeElapsed = new AtomicLong(0);
ContainerManager containerManager = newContainerManager(fakeElapsed);
containerManager.checkContainers();
assertNotNull(zk.exists("/a", false), "node should not have been deleted yet");
+ assertNotNull(zk.exists("/a2", false), "node should not have been deleted yet");
assertNotNull(zk.exists(sequentialPath, false), "node should not have been deleted yet");
+ assertNotNull(zk.exists(sequentialPath2, false), "node should not have been deleted yet");
assertNotNull(zk.exists("/c", false), "node should never be deleted");
fakeElapsed.set(110);
containerManager.checkContainers();
assertNull(zk.exists("/a", false), "node should have been deleted");
+ assertNull(zk.exists("/a2", false), "node should have been deleted");
assertNotNull(zk.exists(sequentialPath, false), "node should not have been deleted yet");
+ assertNotNull(zk.exists(sequentialPath2, false), "node should not have been deleted yet");
assertNotNull(zk.exists("/c", false), "node should never be deleted");
fakeElapsed.set(210);
containerManager.checkContainers();
assertNull(zk.exists("/a", false), "node should have been deleted");
+ assertNull(zk.exists("/a2", false), "node should have been deleted");
assertNull(zk.exists(sequentialPath, false), "node should have been deleted");
+ assertNull(zk.exists(sequentialPath2, false), "node should have been deleted");
assertNotNull(zk.exists("/c", false), "node should never be deleted");
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/MockServerCnxn.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/MockServerCnxn.java
index 4dfcebd84ec..af095921dad 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/MockServerCnxn.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/MockServerCnxn.java
@@ -22,8 +22,10 @@
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.security.cert.Certificate;
+import java.util.List;
import org.apache.jute.Record;
import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.ReplyHeader;
@@ -56,7 +58,7 @@ public void sendCloseSession() {
}
@Override
- public void process(WatchedEvent event) {
+ public void process(WatchedEvent event, List acl) {
}
@Override
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java
index 9e71205697d..e5c2940f39e 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java
@@ -195,7 +195,7 @@ public void testReconfigWithAnotherOutstandingChange() throws Exception {
QuorumPeer qp = new QuorumPeer();
QuorumVerifier quorumVerifierMock = mock(QuorumVerifier.class);
- when(quorumVerifierMock.getAllMembers()).thenReturn(LeaderBeanTest.getMockedPeerViews(qp.getId()));
+ when(quorumVerifierMock.getAllMembers()).thenReturn(LeaderBeanTest.getMockedPeerViews(qp.getMyId()));
qp.setQuorumVerifier(quorumVerifierMock, false);
FileTxnSnapLog snapLog = new FileTxnSnapLog(tmpDir, tmpDir);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java
index 62d371dd4fb..15259207599 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java
@@ -67,11 +67,17 @@ public class RequestThrottlerTest extends ZKTestCase {
CountDownLatch disconnected = null;
+ CountDownLatch throttled = null;
+ CountDownLatch throttling = null;
+
ZooKeeperServer zks = null;
ServerCnxnFactory f = null;
ZooKeeper zk = null;
int connectionLossCount = 0;
+ private long getCounterMetric(String name) {
+ return (long) MetricsUtils.currentServerMetrics().get(name);
+ }
@BeforeEach
public void setup() throws Exception {
@@ -115,6 +121,11 @@ public TestZooKeeperServer(File snapDir, File logDir, int tickTime) throws IOExc
super(snapDir, logDir, tickTime);
}
+ @Override
+ protected RequestThrottler createRequestThrottler() {
+ return new TestRequestThrottler(this);
+ }
+
@Override
protected void setupRequestProcessors() {
RequestProcessor finalProcessor = new FinalRequestProcessor(this);
@@ -141,6 +152,24 @@ public void requestFinished(Request request) {
}
}
+ class TestRequestThrottler extends RequestThrottler {
+ public TestRequestThrottler(ZooKeeperServer zks) {
+ super(zks);
+ }
+
+ @Override
+ synchronized void throttleSleep(int stallTime) throws InterruptedException {
+ if (throttling != null) {
+ throttling.countDown();
+ }
+ super.throttleSleep(stallTime);
+ // Defend against unstable timing and potential spurious wakeup.
+ if (throttled != null) {
+ assertTrue(throttled.await(20, TimeUnit.SECONDS));
+ }
+ }
+ }
+
class TestPrepRequestProcessor extends PrepRequestProcessor {
public TestPrepRequestProcessor(ZooKeeperServer zks, RequestProcessor syncProcessor) {
@@ -191,18 +220,22 @@ public void testRequestThrottler() throws Exception {
// make sure the server received all 5 requests
submitted.await(5, TimeUnit.SECONDS);
- Map metrics = MetricsUtils.currentServerMetrics();
// but only two requests can get into the pipeline because of the throttler
- assertEquals(2L, (long) metrics.get("prep_processor_request_queued"));
- assertEquals(1L, (long) metrics.get("request_throttle_wait_count"));
+ WaitForCondition requestQueued = () -> getCounterMetric("prep_processor_request_queued") == 2;
+ waitFor("request not queued", requestQueued, 5);
+
+ WaitForCondition throttleWait = () -> getCounterMetric("request_throttle_wait_count") >= 1;
+ waitFor("no throttle wait", throttleWait, 5);
// let the requests go through the pipeline and the throttler will be waken up to allow more requests
// to enter the pipeline
resumeProcess.countDown();
- entered.await(STALL_TIME, TimeUnit.MILLISECONDS);
- metrics = MetricsUtils.currentServerMetrics();
+ // wait for more than one STALL_TIME to reduce timeout before wakeup
+ assertTrue(entered.await(STALL_TIME + 5000, TimeUnit.MILLISECONDS));
+
+ Map metrics = MetricsUtils.currentServerMetrics();
assertEquals(TOTAL_REQUESTS, (long) metrics.get("prep_processor_request_queued"));
}
@@ -221,6 +254,9 @@ public void testDropStaleRequests() throws Exception {
resumeProcess = new CountDownLatch(1);
submitted = new CountDownLatch(TOTAL_REQUESTS);
+ throttled = new CountDownLatch(1);
+ throttling = new CountDownLatch(1);
+
// send 5 requests asynchronously
for (int i = 0; i < TOTAL_REQUESTS; i++) {
zk.create("/request_throttle_test- " + i, ("/request_throttle_test- "
@@ -229,18 +265,20 @@ public void testDropStaleRequests() throws Exception {
}
// make sure the server received all 5 requests
- submitted.await(5, TimeUnit.SECONDS);
- Map metrics = MetricsUtils.currentServerMetrics();
-
- // but only two requests can get into the pipeline because of the throttler
- assertEquals(2L, (long) metrics.get("prep_processor_request_queued"));
- assertEquals(1L, (long) metrics.get("request_throttle_wait_count"));
+ assertTrue(submitted.await(5, TimeUnit.SECONDS));
+ // stale throttled requests
+ assertTrue(throttling.await(5, TimeUnit.SECONDS));
for (ServerCnxn cnxn : f.cnxns) {
cnxn.setStale();
}
+ throttled.countDown();
zk = null;
+ // only first three requests are counted as finished
+ finished = new CountDownLatch(3);
+
+ // let the requests go through the pipeline
resumeProcess.countDown();
LOG.info("raise the latch");
@@ -248,10 +286,18 @@ public void testDropStaleRequests() throws Exception {
Thread.sleep(50);
}
+ assertTrue(finished.await(5, TimeUnit.SECONDS));
+
+ // assert after all requests processed to avoid concurrent issues as metrics are
+ // counted in different threads.
+ Map metrics = MetricsUtils.currentServerMetrics();
+
+ // only two requests can get into the pipeline because of the throttler
+ assertEquals(2L, (long) metrics.get("prep_processor_request_queued"));
+
// the rest of the 3 requests will be dropped
// but only the first one for a connection will be counted
- metrics = MetricsUtils.currentServerMetrics();
- assertEquals(2L, (long) metrics.get("prep_processor_request_queued"));
+ assertEquals(1L, (long) metrics.get("request_throttle_wait_count"));
assertEquals(1, (long) metrics.get("stale_requests_dropped"));
}
@@ -261,13 +307,22 @@ public void testLargeRequestThrottling() throws Exception {
AsyncCallback.StringCallback createCallback = (rc, path, ctx, name) -> {
if (KeeperException.Code.get(rc) == KeeperException.Code.CONNECTIONLOSS) {
- disconnected.countDown();
connectionLossCount++;
+ disconnected.countDown();
}
};
- // we allow five requests in the pipeline
- RequestThrottler.setMaxRequests(5);
+ // the total length of the request is about 170-180 bytes, so only two requests are allowed
+ byte[] data = new byte[100];
+ // the third request will incur throttle. We don't send more requests to avoid reconnecting
+ // due to unstable test environment(e.g. slow sending).
+ int number_requests = 3;
+
+ // we allow more requests in the pipeline
+ RequestThrottler.setMaxRequests(number_requests + 2);
+
+ // request could become stale in processor threads due to throttle in io thread
+ RequestThrottler.setDropStaleRequests(false);
// enable large request throttling
zks.setLargeRequestThreshold(150);
@@ -277,34 +332,32 @@ public void testLargeRequestThrottling() throws Exception {
resumeProcess = new CountDownLatch(1);
// the connection will be close when large requests exceed the limit
// we can't use the submitted latch because requests after close won't be submitted
- disconnected = new CountDownLatch(TOTAL_REQUESTS);
-
- // the total length of the request is about 170-180 bytes, so only two requests are allowed
- byte[] data = new byte[100];
+ disconnected = new CountDownLatch(number_requests);
- // send 5 requests asynchronously
- for (int i = 0; i < TOTAL_REQUESTS; i++) {
+ // send requests asynchronously
+ for (int i = 0; i < number_requests; i++) {
zk.create("/request_throttle_test- " + i , data,
ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, createCallback, null);
}
- // make sure the server received all 5 requests
- disconnected.await(30, TimeUnit.SECONDS);
+ // make sure the server received all requests
+ assertTrue(disconnected.await(30, TimeUnit.SECONDS));
+
+ finished = new CountDownLatch(2);
+ // let the requests go through the pipeline
+ resumeProcess.countDown();
+ assertTrue(finished.await(5, TimeUnit.SECONDS));
+
+ // assert metrics after finished so metrics in no io threads are set also.
Map metrics = MetricsUtils.currentServerMetrics();
// but only two requests can get into the pipeline because they are large requests
// the connection will be closed
assertEquals(2L, (long) metrics.get("prep_processor_request_queued"));
assertEquals(1L, (long) metrics.get("large_requests_rejected"));
- assertEquals(5, connectionLossCount);
-
- finished = new CountDownLatch(2);
- // let the requests go through the pipeline
- resumeProcess.countDown();
- finished.await(5, TimeUnit.SECONDS);
+ assertEquals(number_requests, connectionLossCount);
// when the two requests finish, they are stale because the connection is closed already
- metrics = MetricsUtils.currentServerMetrics();
assertEquals(2, (long) metrics.get("stale_replies"));
}
@@ -319,7 +372,6 @@ public void testGlobalOutstandingRequestThrottlingWithRequestThrottlerDisabled()
RequestThrottler.setMaxRequests(0);
resumeProcess = new CountDownLatch(1);
int totalRequests = 10;
- submitted = new CountDownLatch(totalRequests);
for (int i = 0; i < totalRequests; i++) {
zk.create("/request_throttle_test- " + i, ("/request_throttle_test- "
@@ -327,16 +379,16 @@ public void testGlobalOutstandingRequestThrottlingWithRequestThrottlerDisabled()
}, null);
}
- submitted.await(5, TimeUnit.SECONDS);
-
// We should start throttling instead of queuing more requests.
//
// We always allow up to GLOBAL_OUTSTANDING_LIMIT + 1 number of requests coming in request processing pipeline
// before throttling. For the next request, we will throttle by disabling receiving future requests but we still
- // allow this single request coming in. So the total number of queued requests in processing pipeline would
+ // allow this single request coming in. Ideally, the total number of queued requests in processing pipeline would
// be GLOBAL_OUTSTANDING_LIMIT + 2.
- assertEquals(Integer.parseInt(GLOBAL_OUTSTANDING_LIMIT) + 2,
- (long) MetricsUtils.currentServerMetrics().get("prep_processor_request_queued"));
+ //
+ // But due to leak of consistent view of number of outstanding requests, the number could be larger.
+ WaitForCondition requestQueued = () -> getCounterMetric("prep_processor_request_queued") >= Integer.parseInt(GLOBAL_OUTSTANDING_LIMIT) + 2;
+ waitFor("no enough requests queued", requestQueued, 5);
resumeProcess.countDown();
} catch (Exception e) {
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerMainTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerMainTest.java
index 920fdf406bf..0a76cb363c0 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerMainTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerMainTest.java
@@ -198,7 +198,7 @@ public void testNonRecoverableError() throws Exception {
// inject problem in server
ZooKeeperServer zooKeeperServer = main.getCnxnFactory().getZooKeeperServer();
FileTxnSnapLog snapLog = zooKeeperServer.getTxnLogFactory();
- FileTxnSnapLog fileTxnSnapLogWithError = new FileTxnSnapLog(snapLog.getDataDir(), snapLog.getSnapDir()) {
+ FileTxnSnapLog fileTxnSnapLogWithError = new FileTxnSnapLog(snapLog.getDataLogDir(), snapLog.getSnapDir()) {
@Override
public void commit() throws IOException {
throw new IOException("Input/output error");
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerTest.java
index bd141206905..c521deb2b72 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerTest.java
@@ -30,6 +30,7 @@
import java.util.List;
import java.util.Map;
import java.util.UUID;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.metrics.MetricsUtils;
import org.apache.zookeeper.server.persistence.FileTxnLog;
@@ -42,6 +43,45 @@
public class ZooKeeperServerTest extends ZKTestCase {
+ @Test
+ public void testDirSize() throws Exception {
+ ZooKeeperServer zks = null;
+ ServerCnxnFactory cnxnFactory = null;
+
+ try {
+ final File dataDir = ClientBase.createTmpDir();
+ final File logDir = ClientBase.createTmpDir();
+
+ zks = new ZooKeeperServer(dataDir, logDir, 3000);
+
+ // validate dir size before server starts
+ assertEquals(0, zks.getDataDirSize());
+ assertEquals(0, zks.getLogDirSize());
+
+ // start server
+ final String hostPort = "127.0.0.1:" + PortAssignment.unique();
+ final int port = Integer.parseInt(hostPort.split(":")[1]);
+ cnxnFactory = ServerCnxnFactory.createFactory(port, -1);
+ cnxnFactory.startup(zks);
+ assertTrue(ClientBase.waitForServerUp(hostPort, 120000));
+
+ // validate data size is greater than 0 as snapshot has been taken when server starts
+ assertTrue(zks.getDataDirSize() > 0);
+
+ // validate log size is 0 as no txn yet
+ assertEquals(0, zks.getLogDirSize());
+ } finally {
+ if (cnxnFactory != null) {
+ cnxnFactory.shutdown();
+ }
+
+ if (zks != null) {
+ zks.shutdown();
+ }
+ }
+ }
+
+
@Test
public void testSortDataDirAscending() {
File[] files = new File[5];
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogMetricsTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogMetricsTest.java
index a93dcc80b05..65648fefce4 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogMetricsTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogMetricsTest.java
@@ -22,20 +22,21 @@
import static org.hamcrest.number.OrderingComparison.greaterThan;
import static org.hamcrest.number.OrderingComparison.greaterThanOrEqualTo;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import java.io.File;
import java.util.Map;
-import java.util.concurrent.CountDownLatch;
import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.metrics.MetricsUtils;
+import org.apache.zookeeper.server.ServerCnxnFactory;
import org.apache.zookeeper.server.ServerMetrics;
import org.apache.zookeeper.server.SyncRequestProcessor;
+import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.test.ClientBase;
-import org.apache.zookeeper.test.QuorumUtil;
+import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -43,43 +44,53 @@ public class FileTxnSnapLogMetricsTest extends ZKTestCase {
private static final Logger LOG = LoggerFactory.getLogger(FileTxnSnapLogMetricsTest.class);
- CountDownLatch allCreatedLatch;
+ @TempDir
+ File logDir;
- private class MockWatcher implements Watcher {
+ @TempDir
+ File snapDir;
- @Override
- public void process(WatchedEvent e) {
- LOG.info("all nodes created");
- allCreatedLatch.countDown();
- }
+ private ServerCnxnFactory startServer() throws Exception {
+ ZooKeeperServer zkServer = new ZooKeeperServer(snapDir, logDir, 3000);
+ ServerCnxnFactory cnxnFactory = ServerCnxnFactory.createFactory(0, -1);
+ cnxnFactory.startup(zkServer);
+ return cnxnFactory;
+ }
+ @AfterEach
+ public void cleanup() throws Exception {
+ SyncRequestProcessor.setSnapCount(ZooKeeperServer.getSnapCount());
}
@Test
public void testFileTxnSnapLogMetrics() throws Exception {
SyncRequestProcessor.setSnapCount(100);
- QuorumUtil util = new QuorumUtil(1);
- util.startAll();
+ ServerCnxnFactory cnxnFactory = startServer();
+ String connectString = "127.0.0.1:" + cnxnFactory.getLocalPort();
- allCreatedLatch = new CountDownLatch(1);
+ // Snapshot in load data.
+ assertEquals(1L, MetricsUtils.currentServerMetrics().get("cnt_snapshottime"));
byte[] data = new byte[500];
- // make sure a snapshot is taken and some txns are not in a snapshot
- ZooKeeper zk = ClientBase.createZKClient(util.getConnString());
+ ZooKeeper zk = ClientBase.createZKClient(connectString);
for (int i = 0; i < 150; i++) {
zk.create("/path" + i, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
}
- if (null == zk.exists("/path149", new MockWatcher())) {
- allCreatedLatch.await();
- }
+ // It is possible that above writes will trigger more than one snapshot due to randomization.
+ WaitForCondition newSnapshot = () -> (long) MetricsUtils.currentServerMetrics().get("cnt_snapshottime") >= 2L;
+ waitFor("no snapshot in 10s", newSnapshot, 10);
+
+ // Pauses snapshot and logs more txns.
+ cnxnFactory.getZooKeeperServer().getTxnLogFactory().snapLog.close();
+ zk.create("/" + 1000, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/" + 1001, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ // Restart server to count startup metrics.
+ cnxnFactory.shutdown();
ServerMetrics.getMetrics().resetAll();
- int leader = util.getLeaderServer();
- // restart a server so it will read the snapshot and the txn logs
- util.shutdown(leader);
- util.start(leader);
+ cnxnFactory = startServer();
Map values = MetricsUtils.currentServerMetrics();
LOG.info("txn loaded during start up {}", values.get("max_startup_txns_loaded"));
@@ -90,7 +101,7 @@ public void testFileTxnSnapLogMetrics() throws Exception {
assertEquals(1L, values.get("cnt_startup_snap_load_time"));
assertThat((long) values.get("max_startup_snap_load_time"), greaterThan(0L));
- util.shutdownAll();
+ cnxnFactory.shutdown();
}
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java
index 44a00993443..103c832fbf5 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java
@@ -177,7 +177,7 @@ public void testWithAutoCreateDataDir() throws IOException {
assertTrue(logDir.exists());
assertTrue(snapDir.exists());
- assertTrue(fileTxnSnapLog.getDataDir().exists());
+ assertTrue(fileTxnSnapLog.getDataLogDir().exists());
assertTrue(fileTxnSnapLog.getSnapDir().exists());
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java
index e8b19dced0e..ad43d8826b0 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java
@@ -251,7 +251,7 @@ public void testCnxManagerSpinLock() throws Exception {
LOG.error("Null listener when initializing cnx manager");
}
- InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne();
+ InetSocketAddress address = peers.get(peer.getMyId()).electionAddr.getReachableOrOne();
LOG.info("Election port: {}", address.getPort());
Thread.sleep(1000);
@@ -339,7 +339,7 @@ public void testCnxManagerNPE() throws Exception {
} else {
LOG.error("Null listener when initializing cnx manager");
}
- InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne();
+ InetSocketAddress address = peers.get(peer.getMyId()).electionAddr.getReachableOrOne();
LOG.info("Election port: {}", address.getPort());
Thread.sleep(1000);
@@ -386,7 +386,7 @@ public void testSocketTimeout() throws Exception {
} else {
LOG.error("Null listener when initializing cnx manager");
}
- InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne();
+ InetSocketAddress address = peers.get(peer.getMyId()).electionAddr.getReachableOrOne();
LOG.info("Election port: {}", address.getPort());
Thread.sleep(1000);
@@ -503,7 +503,7 @@ public void testWorkerThreads() throws Exception {
try {
for (int sid = 0; sid < 3; sid++) {
QuorumPeer peer = new QuorumPeer(peers, peerTmpdir[sid], peerTmpdir[sid], peerClientPort[sid], 3, sid, 1000, 2, 2, 2);
- LOG.info("Starting peer {}", peer.getId());
+ LOG.info("Starting peer {}", peer.getMyId());
peer.start();
peerList.add(sid, peer);
}
@@ -513,14 +513,14 @@ public void testWorkerThreads() throws Exception {
for (int i = 0; i < 5; i++) {
// halt one of the listeners and verify count
QuorumPeer peer = peerList.get(myid);
- LOG.info("Round {}, halting peer {}", i, peer.getId());
+ LOG.info("Round {}, halting peer {}", i, peer.getMyId());
peer.shutdown();
peerList.remove(myid);
failure = verifyThreadCount(peerList, 2);
assertNull(failure, failure);
// Restart halted node and verify count
peer = new QuorumPeer(peers, peerTmpdir[myid], peerTmpdir[myid], peerClientPort[myid], 3, myid, 1000, 2, 2, 2);
- LOG.info("Round {}, restarting peer {}", i, peer.getId());
+ LOG.info("Round {}, restarting peer {}", i, peer.getMyId());
peer.start();
peerList.add(myid, peer);
failure = verifyThreadCount(peerList, 4);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EagerACLFilterTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EagerACLFilterTest.java
index a27d5cf9f0f..4141c586ff8 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EagerACLFilterTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EagerACLFilterTest.java
@@ -19,14 +19,16 @@
package org.apache.zookeeper.server.quorum;
import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
-import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CompletableFuture;
import java.util.stream.Stream;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.TestableZooKeeper;
import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;
import org.apache.zookeeper.test.QuorumBase;
@@ -38,7 +40,6 @@
public class EagerACLFilterTest extends QuorumBase {
- protected final CountDownLatch callComplete = new CountDownLatch(1);
protected boolean complete = false;
protected static final String PARENT_PATH = "/foo";
protected static final String CHILD_PATH = "/foo/bar";
@@ -48,7 +49,9 @@ public class EagerACLFilterTest extends QuorumBase {
protected static final byte[] DATA = "Hint Water".getBytes();
protected TestableZooKeeper zkClient;
protected TestableZooKeeper zkClientB;
+ protected TestableZooKeeper zkLeaderClient;
protected QuorumPeer zkLeader;
+ protected QuorumPeer zkConnected;
protected ZooKeeperServer connectedServer;
public static Stream data() {
@@ -70,6 +73,7 @@ public void setUp() {
public void setUp(ServerState serverState, boolean checkEnabled) throws Exception {
ensureCheck(checkEnabled);
+ CountdownWatcher leaderWatch = new CountdownWatcher();
CountdownWatcher clientWatch = new CountdownWatcher();
CountdownWatcher clientWatchB = new CountdownWatcher();
super.setUp(true, true);
@@ -78,16 +82,31 @@ public void setUp(ServerState serverState, boolean checkEnabled) throws Exceptio
int clientPort = Integer.parseInt(hostPort.split(":")[1]);
zkLeader = getPeerList().get(getLeaderIndex());
- connectedServer = getPeerByClientPort(clientPort).getActiveServer();
+ zkConnected = getPeerByClientPort(clientPort);
+ connectedServer = zkConnected.getActiveServer();
+ zkLeaderClient = createClient(leaderWatch, getPeersMatching(ServerState.LEADING));
zkClient = createClient(clientWatch, hostPort);
zkClientB = createClient(clientWatchB, hostPort);
zkClient.addAuthInfo(AUTH_PROVIDER, AUTH);
zkClientB.addAuthInfo(AUTH_PROVIDER, AUTHB);
+ leaderWatch.waitForConnected(CONNECTION_TIMEOUT);
clientWatch.waitForConnected(CONNECTION_TIMEOUT);
clientWatchB.waitForConnected(CONNECTION_TIMEOUT);
}
+ void syncClient(ZooKeeper zk) {
+ CompletableFuture synced = new CompletableFuture<>();
+ zk.sync("/", (rc, path, ctx) -> {
+ if (rc == 0) {
+ synced.complete(null);
+ } else {
+ synced.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
+ }
+ }, null);
+ synced.join();
+ }
+
@AfterEach
public void tearDown() throws Exception {
if (zkClient != null) {
@@ -102,19 +121,29 @@ public void tearDown() throws Exception {
}
private void ensureCheck(boolean enabled) {
- if (enabled) {
- System.setProperty(ZooKeeperServer.ENABLE_EAGER_ACL_CHECK, "true");
- } else {
- System.clearProperty(ZooKeeperServer.ENABLE_EAGER_ACL_CHECK);
- }
+ ZooKeeperServer.setEnableEagerACLCheck(enabled);
}
- private void assertTransactionState(String condition, long lastxid, ServerState serverState, boolean checkEnabled) {
- String assertion = String.format("Server State: %s Check Enabled: %s %s", serverState, checkEnabled, condition);
- if (checkEnabled) {
- assertEquals(lastxid, zkLeader.getLastLoggedZxid(), assertion);
+ private void assertTransactionState(String operation, QuorumPeer peer, long lastxid) {
+ if (peer == zkLeader && peer != zkConnected) {
+ // The operation is performed on no leader, but we are asserting on leader.
+ // There is no happen-before between `zkLeader.getLastLoggedZxid()` and
+ // successful response from other server. The commit and response are routed
+ // to different servers and performed asynchronous in each server. So we have
+ // to sync leader client to go through commit and response path in leader to
+ // build happen-before between `zkLeader.getLastLoggedZxid()` and side effect
+ // of previous operation.
+ syncClient(zkLeaderClient);
+ }
+ assertTrue(peer == zkLeader || peer == zkConnected);
+ boolean eagerACL = ZooKeeperServer.isEnableEagerACLCheck();
+ String assertion = String.format(
+ "Connecting: %s Checking: %s EagerACL: %s Operation: %s",
+ zkConnected.getPeerState(), peer.getPeerState(), eagerACL, operation);
+ if (eagerACL) {
+ assertEquals(lastxid, peer.getLastLoggedZxid(), assertion);
} else {
- assertNotSame(lastxid, zkLeader.getLastLoggedZxid(), assertion);
+ assertNotEquals(lastxid, peer.getLastLoggedZxid(), assertion);
}
}
@@ -144,15 +173,17 @@ public void testCreate2OK(ServerState serverState, boolean checkEnabled) throws
public void testCreateFail(ServerState serverState, boolean checkEnabled) throws Exception {
setUp(serverState, checkEnabled);
zkClient.create(PARENT_PATH, DATA, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
- long lastxid = zkLeader.getLastLoggedZxid();
+ long lastxid = zkConnected.getLastLoggedZxid();
try {
zkClientB.create(CHILD_PATH, DATA, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ fail("expect no auth");
} catch (KeeperException.NoAuthException e) {
}
assertEquals(0, connectedServer.getInProcess(), "OutstandingRequests not decremented");
- assertTransactionState("Transaction state on Leader after failed create", lastxid, serverState, checkEnabled);
+ assertTransactionState("failed create", zkConnected, lastxid);
+ assertTransactionState("failed create", zkLeader, lastxid);
}
@ParameterizedTest
@@ -160,15 +191,17 @@ public void testCreateFail(ServerState serverState, boolean checkEnabled) throws
public void testCreate2Fail(ServerState serverState, boolean checkEnabled) throws Exception {
setUp(serverState, checkEnabled);
zkClient.create(PARENT_PATH, DATA, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT, null);
- long lastxid = zkLeader.getLastLoggedZxid();
+ long lastxid = zkConnected.getLastLoggedZxid();
try {
zkClientB.create(CHILD_PATH, DATA, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, null);
+ fail("expect no auth");
} catch (KeeperException.NoAuthException e) {
}
assertEquals(0, connectedServer.getInProcess(), "OutstandingRequests not decremented");
- assertTransactionState("Transaction state on Leader after failed create2", lastxid, serverState, checkEnabled);
+ assertTransactionState("failed create2", zkConnected, lastxid);
+ assertTransactionState("failed create2", zkLeader, lastxid);
}
@ParameterizedTest
@@ -187,15 +220,17 @@ public void testDeleteFail(ServerState serverState, boolean checkEnabled) throws
setUp(serverState, checkEnabled);
zkClient.create(PARENT_PATH, DATA, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT, null);
zkClient.create(CHILD_PATH, DATA, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT, null);
- long lastxid = zkLeader.getLastLoggedZxid();
+ long lastxid = zkConnected.getLastLoggedZxid();
try {
zkClientB.delete(CHILD_PATH, -1);
+ fail("expect no auth");
} catch (KeeperException.NoAuthException e) {
}
assertEquals(0, connectedServer.getInProcess(), "OutstandingRequests not decremented");
- assertTransactionState("Transaction state on Leader after failed delete", lastxid, serverState, checkEnabled);
+ assertTransactionState("failed delete", zkConnected, lastxid);
+ assertTransactionState("failed delete", zkLeader, lastxid);
}
@ParameterizedTest
@@ -211,15 +246,17 @@ public void testSetDataOK(ServerState serverState, boolean checkEnabled) throws
public void testSetDataFail(ServerState serverState, boolean checkEnabled) throws Exception {
setUp(serverState, checkEnabled);
zkClient.create(PARENT_PATH, null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT, null);
- long lastxid = zkLeader.getLastLoggedZxid();
+ long lastxid = zkConnected.getLastLoggedZxid();
try {
zkClientB.setData(PARENT_PATH, DATA, -1);
+ fail("expect no auth");
} catch (KeeperException.NoAuthException e) {
}
assertEquals(0, connectedServer.getInProcess(), "OutstandingRequests not decremented");
- assertTransactionState("Transaction state on Leader after failed setData", lastxid, serverState, checkEnabled);
+ assertTransactionState("failed setData", zkConnected, lastxid);
+ assertTransactionState("failed setData", zkLeader, lastxid);
}
@ParameterizedTest
@@ -237,15 +274,17 @@ public void testSetACLOK(ServerState serverState, boolean checkEnabled) throws E
public void testSetACLFail(ServerState serverState, boolean checkEnabled) throws Exception {
setUp(serverState, checkEnabled);
zkClient.create(PARENT_PATH, null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT, null);
- long lastxid = zkLeader.getLastLoggedZxid();
+ long lastxid = zkConnected.getLastLoggedZxid();
try {
zkClientB.setACL(PARENT_PATH, Ids.READ_ACL_UNSAFE, -1);
- } catch (KeeperException.NoAuthException e) {
+ fail("expect no auth");
+ } catch (KeeperException.NoAuthException ignored) {
}
assertEquals(0, connectedServer.getInProcess(), "OutstandingRequests not decremented");
- assertTransactionState("Transaction state on Leader after failed setACL", lastxid, serverState, checkEnabled);
+ assertTransactionState("failed setACL", zkConnected, lastxid);
+ assertTransactionState("failed setACL", zkLeader, lastxid);
}
@ParameterizedTest
@@ -253,12 +292,12 @@ public void testSetACLFail(ServerState serverState, boolean checkEnabled) throws
public void testBadACL(ServerState serverState, boolean checkEnabled) throws Exception {
setUp(serverState, checkEnabled);
CountdownWatcher cw = new CountdownWatcher();
- TestableZooKeeper zk = createClient(cw, getPeersMatching(serverState));
- long lastxid;
+ String addr = String.format("%s:%d", LOCALADDR, zkConnected.getClientPort());
+ TestableZooKeeper zk = createClient(cw, addr);
cw.waitForConnected(CONNECTION_TIMEOUT);
- lastxid = zkLeader.getLastLoggedZxid();
+ long lastxid = zkConnected.getLastLoggedZxid();
try {
zk.create("/acltest", new byte[0], Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
@@ -268,7 +307,8 @@ public void testBadACL(ServerState serverState, boolean checkEnabled) throws Exc
assertEquals(0, connectedServer.getInProcess(), "OutstandingRequests not decremented");
- assertTransactionState("zxid after invalid ACL", lastxid, serverState, checkEnabled);
+ assertTransactionState("invalid ACL", zkConnected, lastxid);
+ assertTransactionState("invalid ACL", zkLeader, lastxid);
}
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java
index 3dcaf5de1da..c39bc54e0fa 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java
@@ -533,7 +533,7 @@ public void createSessionTracker() {
this,
getZKDatabase().getSessionWithTimeOuts(),
this.tickTime,
- self.getId(),
+ self.getMyId(),
self.areLocalSessionsEnabled(),
getZooKeeperServerListener()) {
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
index 618ffa04f69..6316badb445 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
@@ -25,6 +25,7 @@
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
import static org.mockito.Mockito.when;
import java.io.File;
import java.io.IOException;
@@ -48,6 +49,7 @@
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@@ -76,7 +78,7 @@ public static Map getMockedPeerViews(long myId) {
public void setUp() throws IOException, X509Exception {
qp = new QuorumPeer();
quorumVerifierMock = mock(QuorumVerifier.class);
- when(quorumVerifierMock.getAllMembers()).thenReturn(getMockedPeerViews(qp.getId()));
+ when(quorumVerifierMock.getAllMembers()).thenReturn(getMockedPeerViews(qp.getMyId()));
qp.setQuorumVerifier(quorumVerifierMock, false);
File tmpDir = ClientBase.createEmptyTestDir();
@@ -93,6 +95,15 @@ public void tearDown() throws IOException {
fileTxnSnapLog.close();
}
+ @Test
+ public void testCreateServerSocketWillRecreateInetSocketAddr() {
+ Leader spyLeader = Mockito.spy(leader);
+ InetSocketAddress addr = new InetSocketAddress("localhost", PortAssignment.unique());
+ spyLeader.createServerSocket(addr, false, false);
+ // make sure the address to be bound will be recreated with expected hostString and port
+ Mockito.verify(spyLeader, times(1)).recreateInetSocketAddr(addr.getHostString(), addr.getPort());
+ }
+
@Test
public void testGetName() {
assertEquals("Leader", leaderBean.getName());
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java
index f5df549c51b..7ac563698b5 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java
@@ -73,7 +73,7 @@ public void testGetEpochToPropose() throws Exception {
try {
// Leader asks for epoch (mocking Leader.lead behavior)
// First add to connectingFollowers
- leader.getEpochToPropose(peer.getId(), lastAcceptedEpoch);
+ leader.getEpochToPropose(peer.getMyId(), lastAcceptedEpoch);
} catch (InterruptedException e) {
// ignore timeout
}
@@ -113,7 +113,7 @@ public void testWaitForEpochAck() throws Exception {
assertFalse(leader.electionFinished);
try {
// leader calls waitForEpochAck, first add to electingFollowers
- leader.waitForEpochAck(peer.getId(), new StateSummary(0, 0));
+ leader.waitForEpochAck(peer.getMyId(), new StateSummary(0, 0));
} catch (InterruptedException e) {
// ignore timeout
}
@@ -152,7 +152,7 @@ public void testWaitForNewLeaderAck() throws Exception {
assertFalse(leader.quorumFormed);
try {
// leader calls waitForNewLeaderAck, first add to ackSet
- leader.waitForNewLeaderAck(peer.getId(), zxid);
+ leader.waitForNewLeaderAck(peer.getMyId(), zxid);
} catch (InterruptedException e) {
// ignore timeout
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java
index 6ff491a6aa9..74d4370732e 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java
@@ -85,7 +85,7 @@ public void testClientAddress() throws Exception {
public void testLocalPeerIsLeader() throws Exception {
long localPeerId = 7;
QuorumPeer peer = mock(QuorumPeer.class);
- when(peer.getId()).thenReturn(localPeerId);
+ when(peer.getMyId()).thenReturn(localPeerId);
when(peer.isLeader(eq(localPeerId))).thenReturn(true);
LocalPeerBean localPeerBean = new LocalPeerBean(peer);
assertTrue(localPeerBean.isLeader());
@@ -95,7 +95,7 @@ public void testLocalPeerIsLeader() throws Exception {
public void testLocalPeerIsNotLeader() throws Exception {
long localPeerId = 7;
QuorumPeer peer = mock(QuorumPeer.class);
- when(peer.getId()).thenReturn(localPeerId);
+ when(peer.getMyId()).thenReturn(localPeerId);
when(peer.isLeader(eq(localPeerId))).thenReturn(false);
LocalPeerBean localPeerBean = new LocalPeerBean(peer);
assertFalse(localPeerBean.isLeader());
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumBeanTest.java
index 67bb673f113..62847461d16 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumBeanTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumBeanTest.java
@@ -29,7 +29,7 @@ public class QuorumBeanTest {
@Test
public void testGetNameProperty() {
QuorumPeer qpMock = mock(QuorumPeer.class);
- when(qpMock.getId()).thenReturn(1L);
+ when(qpMock.getMyId()).thenReturn(1L);
QuorumBean qb = new QuorumBean(qpMock);
assertThat("getName property should return Bean name in the right format", qb.getName(), equalTo("ReplicatedServer_id1"));
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCanonicalizeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCanonicalizeTest.java
index 07445f3253f..aec017a0833 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCanonicalizeTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCanonicalizeTest.java
@@ -19,13 +19,18 @@
package org.apache.zookeeper.server.quorum;
import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
import java.net.InetAddress;
import java.net.InetSocketAddress;
+import java.util.LinkedHashMap;
+import java.util.Map;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
+import org.burningwave.tools.net.DefaultHostResolver;
+import org.burningwave.tools.net.HostResolutionRequestInterceptor;
+import org.burningwave.tools.net.MappedHostResolver;
+import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
public class QuorumCanonicalizeTest extends ZKTestCase {
@@ -35,16 +40,28 @@ public class QuorumCanonicalizeTest extends ZKTestCase {
private static final String ZK1_ALIAS = "zookeeper.invalid";
private static final String ZK1_FQDN = "zk1.invalid";
private static final String ZK1_IP = "169.254.0.42";
- private static final InetAddress IA_MOCK_ZK1;
+ private static InetAddress IA_MOCK_ZK1;
- static {
- InetAddress ia = mock(InetAddress.class);
+ @BeforeAll
+ public static void setupDNSMocks() throws Exception {
+ Map hostAliases = new LinkedHashMap<>();
+ hostAliases.put(ZK1_FQDN, ZK1_IP);
+ hostAliases.put(ZK1_ALIAS, ZK1_IP);
- when(ia.getCanonicalHostName()).thenReturn(ZK1_FQDN);
+ HostResolutionRequestInterceptor.INSTANCE.install(
+ new MappedHostResolver(hostAliases),
+ DefaultHostResolver.INSTANCE
+ );
+ InetAddress ia = InetAddress.getByName(ZK1_FQDN);
IA_MOCK_ZK1 = ia;
}
+ @AfterAll
+ public static void clearDNSMocks() {
+ HostResolutionRequestInterceptor.INSTANCE.uninstall();
+ }
+
private static InetAddress getInetAddress(InetSocketAddress addr) {
if (addr.getHostName().equals(ZK1_ALIAS) || addr.getHostName().equals(ZK1_IP)) {
return IA_MOCK_ZK1;
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
index be81aa6c5b3..1cb2a2b8efa 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
@@ -935,7 +935,7 @@ public void testDataDirAndDataLogDir() throws Exception {
// Assert
FileTxnSnapLog txnFactory = qpMain.getQuorumPeer().getTxnFactory();
- assertEquals(Paths.get(dataLogDir.getAbsolutePath(), "version-2").toString(), txnFactory.getDataDir().getAbsolutePath());
+ assertEquals(Paths.get(dataLogDir.getAbsolutePath(), "version-2").toString(), txnFactory.getDataLogDir().getAbsolutePath());
assertEquals(Paths.get(dataDir.getAbsolutePath(), "version-2").toString(), txnFactory.getSnapDir().getAbsolutePath());
} finally {
FileUtils.deleteDirectory(dataDir);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
index 2a116dce076..c36122713fd 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
@@ -356,7 +356,7 @@ public boolean isAlive() {
}
public void reinitialize() throws IOException {
- File dataDir = main.quorumPeer.getTxnFactory().getDataDir();
+ File dataDir = main.quorumPeer.getTxnFactory().getDataLogDir();
ClientBase.recursiveDelete(dataDir);
ClientBase.createInitializeFile(dataDir.getParentFile());
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java
index b5af84f1e60..44d4a3af7b6 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java
@@ -32,6 +32,8 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
import java.math.BigInteger;
import java.net.InetSocketAddress;
import java.net.URLDecoder;
@@ -116,11 +118,24 @@
import org.bouncycastle.util.io.pem.PemWriter;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
public class QuorumSSLTest extends QuorumPeerTestBase {
+ @Retention(RetentionPolicy.RUNTIME)
+ @ParameterizedTest(name = "fipsEnabled = {0}")
+ @ValueSource(booleans = { false, true})
+ private @interface TestBothFipsModes {
+ }
+
+ @Retention(RetentionPolicy.RUNTIME)
+ @ParameterizedTest(name = "fipsEnabled = {0}")
+ @ValueSource(booleans = { false })
+ private @interface TestNoFipsOnly {
+ }
+
private static final String SSL_QUORUM_ENABLED = "sslQuorum=true\n";
private static final String PORT_UNIFICATION_ENABLED = "portUnification=true\n";
private static final String PORT_UNIFICATION_DISABLED = "portUnification=false\n";
@@ -478,9 +493,11 @@ private void clearSSLSystemProperties() {
System.clearProperty(quorumX509Util.getSslProtocolProperty());
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testQuorumSSL() throws Exception {
+ public void testQuorumSSL(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
@@ -499,9 +516,11 @@ public void testQuorumSSL() throws Exception {
assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testQuorumSSL_withPasswordFromFile() throws Exception {
+ public void testQuorumSSL_withPasswordFromFile(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
final Path secretFile = SecretUtilsTest.createSecretFile(String.valueOf(PASSWORD));
System.clearProperty(quorumX509Util.getSslKeystorePasswdProperty());
@@ -523,9 +542,11 @@ public void testQuorumSSL_withPasswordFromFile() throws Exception {
assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testQuorumSSLWithMultipleAddresses() throws Exception {
+ public void testQuorumSSLWithMultipleAddresses(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true");
quorumConfiguration = generateMultiAddressQuorumConfiguration();
@@ -548,9 +569,11 @@ public void testQuorumSSLWithMultipleAddresses() throws Exception {
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testRollingUpgrade() throws Exception {
+ public void testRollingUpgrade(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
// Form a quorum without ssl
q1 = new MainThread(1, clientPortQp1, quorumConfiguration);
q2 = new MainThread(2, clientPortQp2, quorumConfiguration);
@@ -596,9 +619,10 @@ private void stopAppendConfigRestartAll(Map members, String
}
}
- @Test
+ @TestNoFipsOnly
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testHostnameVerificationWithInvalidHostname() throws Exception {
+ public void testHostnameVerificationWithInvalidHostname(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
String badhostnameKeystorePath = tmpDir + "/badhost.jks";
X509Certificate badHostCert = buildEndEntityCert(
defaultKeyPair,
@@ -613,9 +637,10 @@ public void testHostnameVerificationWithInvalidHostname() throws Exception {
testHostnameVerification(badhostnameKeystorePath, false);
}
- @Test
+ @TestNoFipsOnly
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testHostnameVerificationWithInvalidIPAddress() throws Exception {
+ public void testHostnameVerificationWithInvalidIPAddress(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
String badhostnameKeystorePath = tmpDir + "/badhost.jks";
X509Certificate badHostCert = buildEndEntityCert(
defaultKeyPair,
@@ -630,9 +655,11 @@ public void testHostnameVerificationWithInvalidIPAddress() throws Exception {
testHostnameVerification(badhostnameKeystorePath, false);
}
- @Test
+ @TestNoFipsOnly
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testHostnameVerificationWithInvalidIpAddressAndInvalidHostname() throws Exception {
+ public void testHostnameVerificationWithInvalidIpAddressAndInvalidHostname(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
String badhostnameKeystorePath = tmpDir + "/badhost.jks";
X509Certificate badHostCert = buildEndEntityCert(
defaultKeyPair,
@@ -647,9 +674,11 @@ public void testHostnameVerificationWithInvalidIpAddressAndInvalidHostname() thr
testHostnameVerification(badhostnameKeystorePath, false);
}
- @Test
+ @TestNoFipsOnly
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testHostnameVerificationForInvalidMultiAddressServerConfig() throws Exception {
+ public void testHostnameVerificationForInvalidMultiAddressServerConfig(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true");
quorumConfiguration = generateMultiAddressQuorumConfiguration();
@@ -667,9 +696,11 @@ public void testHostnameVerificationForInvalidMultiAddressServerConfig() throws
testHostnameVerification(badhostnameKeystorePath, false);
}
- @Test
+ @TestNoFipsOnly
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testHostnameVerificationWithInvalidIpAddressAndValidHostname() throws Exception {
+ public void testHostnameVerificationWithInvalidIpAddressAndValidHostname(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
String badhostnameKeystorePath = tmpDir + "/badhost.jks";
X509Certificate badHostCert = buildEndEntityCert(
defaultKeyPair,
@@ -684,9 +715,11 @@ public void testHostnameVerificationWithInvalidIpAddressAndValidHostname() throw
testHostnameVerification(badhostnameKeystorePath, true);
}
- @Test
+ @TestNoFipsOnly
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testHostnameVerificationWithValidIpAddressAndInvalidHostname() throws Exception {
+ public void testHostnameVerificationWithValidIpAddressAndInvalidHostname(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
String badhostnameKeystorePath = tmpDir + "/badhost.jks";
X509Certificate badHostCert = buildEndEntityCert(
defaultKeyPair,
@@ -751,9 +784,11 @@ private void testHostnameVerification(String keystorePath, boolean expectSuccess
ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testCertificateRevocationList() throws Exception {
+ public void testCertificateRevocationList(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
@@ -817,9 +852,11 @@ public void testCertificateRevocationList() throws Exception {
assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testOCSP() throws Exception {
+ public void testOCSP(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
Integer ocspPort = PortAssignment.unique();
q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
@@ -891,9 +928,11 @@ public void testOCSP() throws Exception {
}
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testCipherSuites() throws Exception {
+ public void testCipherSuites(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+
// Get default cipher suites from JDK
SSLServerSocketFactory ssf = (SSLServerSocketFactory) SSLServerSocketFactory.getDefault();
List defaultCiphers = new ArrayList();
@@ -932,9 +971,10 @@ public void testCipherSuites() throws Exception {
assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
}
- @Test
+ @TestBothFipsModes
@Timeout(value = 5, unit = TimeUnit.MINUTES)
- public void testProtocolVersion() throws Exception {
+ public void testProtocolVersion(boolean fipsEnabled) throws Exception {
+ System.setProperty(quorumX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
System.setProperty(quorumX509Util.getSslProtocolProperty(), "TLSv1.2");
q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java
index 70f1844eac6..c6b8160a08a 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java
@@ -102,7 +102,7 @@ public Boolean answer(InvocationOnMock invocation) throws Throwable {
QuorumPeer quorumPeer = mock(QuorumPeer.class);
FileTxnSnapLog logfactory = mock(FileTxnSnapLog.class);
// Directories are not used but we need it to avoid NPE
- when(logfactory.getDataDir()).thenReturn(new File(""));
+ when(logfactory.getDataLogDir()).thenReturn(new File(""));
when(logfactory.getSnapDir()).thenReturn(new File(""));
FollowerZooKeeperServer fzks = null;
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java
index 3bdbcd908dc..d6e96716a54 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java
@@ -162,7 +162,7 @@ public void testLeaderInConnectingFollowers() throws Exception {
// epoch were accounted for, so the leader should not block and since it started with
// accepted epoch = 5 it should now have 6
try {
- long epoch = leader.getEpochToPropose(leader.self.getId(), leader.self.getAcceptedEpoch());
+ long epoch = leader.getEpochToPropose(leader.self.getMyId(), leader.self.getAcceptedEpoch());
assertEquals(6, epoch, "leader got wrong epoch from getEpochToPropose");
} catch (Exception e) {
fail("leader timed out in getEpochToPropose");
@@ -575,7 +575,7 @@ public void converseWithFollower(InputArchive ia, OutputArchive oa, Follower f)
File tmpDir = File.createTempFile("test", "dir", testData);
tmpDir.delete();
tmpDir.mkdir();
- File logDir = f.fzk.getTxnLogFactory().getDataDir().getParentFile();
+ File logDir = f.fzk.getTxnLogFactory().getDataLogDir().getParentFile();
File snapDir = f.fzk.getTxnLogFactory().getSnapDir().getParentFile();
//Spy on ZK so we can check if a snapshot happened or not.
f.zk = spy(f.zk);
@@ -709,7 +709,7 @@ public void converseWithFollower(InputArchive ia, OutputArchive oa, Follower f)
File tmpDir = File.createTempFile("test", "dir", testData);
tmpDir.delete();
tmpDir.mkdir();
- File logDir = f.fzk.getTxnLogFactory().getDataDir().getParentFile();
+ File logDir = f.fzk.getTxnLogFactory().getDataLogDir().getParentFile();
File snapDir = f.fzk.getTxnLogFactory().getSnapDir().getParentFile();
//Spy on ZK so we can check if a snapshot happened or not.
f.zk = spy(f.zk);
@@ -940,7 +940,7 @@ public void converseWithObserver(InputArchive ia, OutputArchive oa, Observer o)
File tmpDir = File.createTempFile("test", "dir", testData);
tmpDir.delete();
tmpDir.mkdir();
- File logDir = o.zk.getTxnLogFactory().getDataDir().getParentFile();
+ File logDir = o.zk.getTxnLogFactory().getDataLogDir().getParentFile();
File snapDir = o.zk.getTxnLogFactory().getSnapDir().getParentFile();
try {
assertEquals(0, o.self.getAcceptedEpoch());
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
index 755712b79f9..db95d8d6d30 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
@@ -19,11 +19,17 @@
package org.apache.zookeeper.server.quorum.auth;
import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
import java.util.UUID;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
import org.apache.zookeeper.util.SecurityUtils;
public class KerberosTestUtils {
+ private static final boolean IBM_JAVA = System.getProperty("java.vendor").contains("IBM");
+
private static String keytabFile = new File(System.getProperty("build.test.dir", "build"), UUID.randomUUID().toString()).getAbsolutePath();
public static String getRealm() {
@@ -81,4 +87,59 @@ public static String replacePattern(String[] components, String hostname) {
}
}
+ public static class KerberosConfiguration extends Configuration {
+
+ private String principal;
+ private String keytab;
+ private boolean isInitiator;
+
+ private KerberosConfiguration(String principal, File keytab, boolean client) {
+ this.principal = principal;
+ this.keytab = keytab.getAbsolutePath();
+ this.isInitiator = client;
+ }
+
+ public static Configuration createClientConfig(String principal, File keytab) {
+ return new KerberosConfiguration(principal, keytab, true);
+ }
+
+ public static Configuration createServerConfig(String principal, File keytab) {
+ return new KerberosConfiguration(principal, keytab, false);
+ }
+
+ private static String getKrb5LoginModuleName() {
+ return System.getProperty("java.vendor").contains("IBM")
+ ? "com.ibm.security.auth.module.Krb5LoginModule"
+ : "com.sun.security.auth.module.Krb5LoginModule";
+ }
+
+ @Override
+ public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+ Map options = new HashMap();
+ options.put("principal", principal);
+ options.put("refreshKrb5Config", "true");
+ if (IBM_JAVA) {
+ options.put("useKeytab", keytab);
+ options.put("credsType", "both");
+ } else {
+ options.put("keyTab", keytab);
+ options.put("useKeyTab", "true");
+ options.put("storeKey", "true");
+ options.put("doNotPrompt", "true");
+ options.put("useTicketCache", "true");
+ options.put("renewTGT", "true");
+ options.put("isInitiator", Boolean.toString(isInitiator));
+ }
+ String ticketCache = System.getenv("KRB5CCNAME");
+ if (ticketCache != null) {
+ options.put("ticketCache", ticketCache);
+ }
+ options.put("debug", "true");
+
+ return new AppConfigurationEntry[]{new AppConfigurationEntry(getKrb5LoginModuleName(), AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options)};
+ }
+
+ }
+
+
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java
index 4adcc0b011a..fdd68b5bc83 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java
@@ -63,6 +63,7 @@
* kdc.port=0 (ephemeral port)
* instance=DefaultKrbServer
* max.ticket.lifetime=86400000 (1 day)
+ * min.ticket.lifetime=3600000 (1 hour)
* max.renewable.lifetime=604800000 (7 days)
* transport=TCP
* debug=false
@@ -148,6 +149,7 @@ public void run() {
public static final String KDC_PORT = "kdc.port";
public static final String INSTANCE = "instance";
public static final String MAX_TICKET_LIFETIME = "max.ticket.lifetime";
+ public static final String MIN_TICKET_LIFETIME = "min.ticket.lifetime";
public static final String MAX_RENEWABLE_LIFETIME = "max.renewable.lifetime";
public static final String TRANSPORT = "transport";
public static final String DEBUG = "debug";
@@ -159,11 +161,11 @@ public void run() {
PROPERTIES.add(ORG_NAME);
PROPERTIES.add(ORG_DOMAIN);
PROPERTIES.add(KDC_BIND_ADDRESS);
- PROPERTIES.add(KDC_BIND_ADDRESS);
PROPERTIES.add(KDC_PORT);
PROPERTIES.add(INSTANCE);
PROPERTIES.add(TRANSPORT);
PROPERTIES.add(MAX_TICKET_LIFETIME);
+ PROPERTIES.add(MIN_TICKET_LIFETIME);
PROPERTIES.add(MAX_RENEWABLE_LIFETIME);
DEFAULT_CONFIG.setProperty(KDC_BIND_ADDRESS, "localhost");
@@ -173,6 +175,7 @@ public void run() {
DEFAULT_CONFIG.setProperty(ORG_DOMAIN, "COM");
DEFAULT_CONFIG.setProperty(TRANSPORT, "TCP");
DEFAULT_CONFIG.setProperty(MAX_TICKET_LIFETIME, "86400000");
+ DEFAULT_CONFIG.setProperty(MIN_TICKET_LIFETIME, "3600000");
DEFAULT_CONFIG.setProperty(MAX_RENEWABLE_LIFETIME, "604800000");
DEFAULT_CONFIG.setProperty(DEBUG, "false");
}
@@ -313,6 +316,10 @@ private void prepareKdcServer() throws Exception {
throw new IllegalArgumentException("Need to set transport!");
}
simpleKdc.getKdcConfig().setString(KdcConfigKey.KDC_SERVICE_NAME, conf.getProperty(INSTANCE));
+ long minTicketLifetimeConf = Long.parseLong(conf.getProperty(MIN_TICKET_LIFETIME)) / 1000;
+ simpleKdc.getKdcConfig().setLong(KdcConfigKey.MINIMUM_TICKET_LIFETIME, minTicketLifetimeConf);
+ long maxTicketLifetimeConf = Long.parseLong(conf.getProperty(MAX_TICKET_LIFETIME)) / 1000;
+ simpleKdc.getKdcConfig().setLong(KdcConfigKey.MAXIMUM_TICKET_LIFETIME, maxTicketLifetimeConf);
if (conf.getProperty(DEBUG) != null) {
krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG, conf.getProperty(DEBUG));
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java
index 4ba6edcefaa..3e57f9a4663 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java
@@ -23,18 +23,15 @@
import java.io.File;
import java.security.Principal;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
-import java.util.Map;
import java.util.Set;
import javax.security.auth.Subject;
import javax.security.auth.kerberos.KerberosPrincipal;
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.Configuration;
import javax.security.auth.login.LoginContext;
import org.apache.kerby.kerberos.kerb.keytab.Keytab;
import org.apache.kerby.kerberos.kerb.type.base.PrincipalName;
+import org.apache.zookeeper.server.quorum.auth.KerberosTestUtils.KerberosConfiguration;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
@@ -47,7 +44,6 @@
*/
public class MiniKdcTest extends KerberosSecurityTestcase {
- private static final boolean IBM_JAVA = System.getProperty("java.vendor").contains("IBM");
@Test
@Timeout(value = 60)
@@ -74,59 +70,6 @@ public void testKeytabGen() throws Exception {
principals);
}
- private static class KerberosConfiguration extends Configuration {
-
- private String principal;
- private String keytab;
- private boolean isInitiator;
-
- private KerberosConfiguration(String principal, File keytab, boolean client) {
- this.principal = principal;
- this.keytab = keytab.getAbsolutePath();
- this.isInitiator = client;
- }
-
- public static Configuration createClientConfig(String principal, File keytab) {
- return new KerberosConfiguration(principal, keytab, true);
- }
-
- public static Configuration createServerConfig(String principal, File keytab) {
- return new KerberosConfiguration(principal, keytab, false);
- }
-
- private static String getKrb5LoginModuleName() {
- return System.getProperty("java.vendor").contains("IBM")
- ? "com.ibm.security.auth.module.Krb5LoginModule"
- : "com.sun.security.auth.module.Krb5LoginModule";
- }
-
- @Override
- public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
- Map options = new HashMap();
- options.put("principal", principal);
- options.put("refreshKrb5Config", "true");
- if (IBM_JAVA) {
- options.put("useKeytab", keytab);
- options.put("credsType", "both");
- } else {
- options.put("keyTab", keytab);
- options.put("useKeyTab", "true");
- options.put("storeKey", "true");
- options.put("doNotPrompt", "true");
- options.put("useTicketCache", "true");
- options.put("renewTGT", "true");
- options.put("isInitiator", Boolean.toString(isInitiator));
- }
- String ticketCache = System.getenv("KRB5CCNAME");
- if (ticketCache != null) {
- options.put("ticketCache", ticketCache);
- }
- options.put("debug", "true");
-
- return new AppConfigurationEntry[]{new AppConfigurationEntry(getKrb5LoginModuleName(), AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options)};
- }
-
- }
@Test
@Timeout(value = 60)
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java
index d9a32a7559b..e962cd6a1b6 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java
@@ -57,7 +57,7 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
+ " debug=false\n"
+ " refreshKrb5Config=true\n"
+ " principal=\""
- + KerberosTestUtils.getServerPrincipal()
+ + KerberosTestUtils.replaceHostPattern(KerberosTestUtils.getHostServerPrincipal())
+ "\";\n"
+ "};\n"
+ "QuorumLearner {\n"
@@ -71,7 +71,7 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
+ " debug=false\n"
+ " refreshKrb5Config=true\n"
+ " principal=\""
- + KerberosTestUtils.getLearnerPrincipal()
+ + KerberosTestUtils.replaceHostPattern(KerberosTestUtils.getHostLearnerPrincipal())
+ "\";\n"
+ "};\n";
setupJaasConfig(jaasEntries);
@@ -81,10 +81,10 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
public static void setUp() throws Exception {
// create keytab
keytabFile = new File(KerberosTestUtils.getKeytabFile());
- String learnerPrincipal = KerberosTestUtils.getLearnerPrincipal();
- String serverPrincipal = KerberosTestUtils.getServerPrincipal();
- learnerPrincipal = learnerPrincipal.substring(0, learnerPrincipal.lastIndexOf("@"));
- serverPrincipal = serverPrincipal.substring(0, serverPrincipal.lastIndexOf("@"));
+ String learnerPrincipal = KerberosTestUtils.getHostLearnerPrincipal();
+ String serverPrincipal = KerberosTestUtils.getHostServerPrincipal();
+ learnerPrincipal = KerberosTestUtils.replaceHostPattern(learnerPrincipal.substring(0, learnerPrincipal.lastIndexOf("@")));
+ serverPrincipal = KerberosTestUtils.replaceHostPattern(serverPrincipal.substring(0, serverPrincipal.lastIndexOf("@")));
getKdc().createPrincipal(keytabFile, learnerPrincipal, serverPrincipal);
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java
index f02da0b205b..9e5e76d67f8 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java
@@ -44,15 +44,17 @@ public class QuorumKerberosHostBasedAuthTest extends KerberosSecurityTestcase {
private static String hostServerPrincipal = KerberosTestUtils.getHostServerPrincipal();
private static String hostLearnerPrincipal = KerberosTestUtils.getHostLearnerPrincipal();
private static String hostNamedLearnerPrincipal = KerberosTestUtils.getHostNamedLearnerPrincipal("myHost");
+ private static String hostlessLearnerPrincipal = KerberosTestUtils.getLearnerPrincipal();
static {
- setupJaasConfigEntries(hostServerPrincipal, hostLearnerPrincipal, hostNamedLearnerPrincipal);
+ setupJaasConfigEntries(hostServerPrincipal, hostLearnerPrincipal, hostNamedLearnerPrincipal, hostlessLearnerPrincipal);
}
private static void setupJaasConfigEntries(
String hostServerPrincipal,
String hostLearnerPrincipal,
- String hostNamedLearnerPrincipal) {
+ String hostNamedLearnerPrincipal,
+ String hostlessLearnerPrincipal) {
String keytabFilePath = FilenameUtils.normalize(KerberosTestUtils.getKeytabFile(), true);
// note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM,
@@ -93,6 +95,18 @@ private static void setupJaasConfigEntries(
+ " refreshKrb5Config=true\n"
+ " principal=\"" + hostNamedLearnerPrincipal
+ "\";\n"
+ + "};\n"
+ + "QuorumLearnerMissingHost {\n"
+ + " com.sun.security.auth.module.Krb5LoginModule required\n"
+ + " useKeyTab=true\n"
+ + " keyTab=\"" + keytabFilePath
+ + "\"\n"
+ + " storeKey=true\n"
+ + " useTicketCache=false\n"
+ + " debug=false\n"
+ + " refreshKrb5Config=true\n"
+ + " principal=\"" + hostlessLearnerPrincipal
+ + "\";\n"
+ "};\n";
setupJaasConfig(jaasEntries);
}
@@ -110,7 +124,11 @@ public static void setUp() throws Exception {
// learner with ipaddress in principal
String learnerPrincipal2 = hostNamedLearnerPrincipal.substring(0, hostNamedLearnerPrincipal.lastIndexOf("@"));
- getKdc().createPrincipal(keytabFile, learnerPrincipal, learnerPrincipal2, serverPrincipal);
+
+ // learner without host in principal
+ String learnerPrincipal3 = hostlessLearnerPrincipal.substring(0, hostlessLearnerPrincipal.lastIndexOf("@"));
+
+ getKdc().createPrincipal(keytabFile, learnerPrincipal, learnerPrincipal2, learnerPrincipal3, serverPrincipal);
}
@AfterEach
@@ -224,4 +242,52 @@ public void testConnectBadServer() throws Exception {
}
}
+ /**
+ * Test to verify that the bad server connection to the quorum should be rejected.
+ */
+ @Test
+ @Timeout(value = 120)
+ public void testConnectHostlessPrincipalBadServer() throws Exception {
+ String serverPrincipal = hostServerPrincipal.substring(0, hostServerPrincipal.lastIndexOf("@"));
+ Map authConfigs = new HashMap<>();
+ authConfigs.put(QuorumAuth.QUORUM_SASL_AUTH_ENABLED, "true");
+ authConfigs.put(QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED, "true");
+ authConfigs.put(QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED, "true");
+ authConfigs.put(QuorumAuth.QUORUM_KERBEROS_SERVICE_PRINCIPAL, serverPrincipal);
+ String connectStr = startQuorum(3, authConfigs, 3);
+ CountdownWatcher watcher = new CountdownWatcher();
+ ZooKeeper zk = new ZooKeeper(connectStr, ClientBase.CONNECTION_TIMEOUT, watcher);
+ watcher.waitForConnected(ClientBase.CONNECTION_TIMEOUT);
+ for (int i = 0; i < 10; i++) {
+ zk.create("/" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ }
+ zk.close();
+
+ String quorumCfgSection = mt.get(0).getQuorumCfgSection();
+ StringBuilder sb = new StringBuilder();
+ sb.append(quorumCfgSection);
+
+ int myid = mt.size() + 1;
+ final int clientPort = PortAssignment.unique();
+ String server = String.format("server.%d=localhost:%d:%d:participant", myid, PortAssignment.unique(), PortAssignment.unique());
+ sb.append(server + "\n");
+ quorumCfgSection = sb.toString();
+ authConfigs.put(QuorumAuth.QUORUM_LEARNER_SASL_LOGIN_CONTEXT, "QuorumLearnerMissingHost");
+ MainThread badServer = new MainThread(myid, clientPort, quorumCfgSection, authConfigs);
+ badServer.start();
+ watcher = new CountdownWatcher();
+ connectStr = "127.0.0.1:" + clientPort;
+ zk = new ZooKeeper(connectStr, ClientBase.CONNECTION_TIMEOUT, watcher);
+ try {
+ watcher.waitForConnected(ClientBase.CONNECTION_TIMEOUT / 3);
+ fail("Must throw exception as the principal does not include an authorized host!");
+ } catch (TimeoutException e) {
+ // expected
+ } finally {
+ zk.close();
+ badServer.shutdown();
+ badServer.deleteBaseDir();
+ }
+ }
+
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java
index bd613a44488..83f3538045f 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java
@@ -131,7 +131,7 @@ public WatcherTriggerWorker(
public void run() {
while (!stopped) {
String path = PATH_PREFIX + r.nextInt(paths);
- WatcherOrBitSet s = manager.triggerWatch(path, EventType.NodeDeleted);
+ WatcherOrBitSet s = manager.triggerWatch(path, EventType.NodeDeleted, null);
if (s != null) {
triggeredCount.addAndGet(s.size());
}
@@ -437,20 +437,20 @@ public void testWatcherMetrics(String className) throws IOException {
//path2 is watched by watcher1
manager.addWatch(path2, watcher1);
- manager.triggerWatch(path3, EventType.NodeCreated);
+ manager.triggerWatch(path3, EventType.NodeCreated, null);
//path3 is not being watched so metric is 0
checkMetrics("node_created_watch_count", 0L, 0L, 0D, 0L, 0L);
//path1 is watched by two watchers so two fired
- manager.triggerWatch(path1, EventType.NodeCreated);
+ manager.triggerWatch(path1, EventType.NodeCreated, null);
checkMetrics("node_created_watch_count", 2L, 2L, 2D, 1L, 2L);
//path2 is watched by one watcher so one fired now total is 3
- manager.triggerWatch(path2, EventType.NodeCreated);
+ manager.triggerWatch(path2, EventType.NodeCreated, null);
checkMetrics("node_created_watch_count", 1L, 2L, 1.5D, 2L, 3L);
//watches on path1 are no longer there so zero fired
- manager.triggerWatch(path1, EventType.NodeDataChanged);
+ manager.triggerWatch(path1, EventType.NodeDataChanged, null);
checkMetrics("node_changed_watch_count", 0L, 0L, 0D, 0L, 0L);
//both wather1 and wather2 are watching path1
@@ -460,10 +460,10 @@ public void testWatcherMetrics(String className) throws IOException {
//path2 is watched by watcher1
manager.addWatch(path2, watcher1);
- manager.triggerWatch(path1, EventType.NodeDataChanged);
+ manager.triggerWatch(path1, EventType.NodeDataChanged, null);
checkMetrics("node_changed_watch_count", 2L, 2L, 2D, 1L, 2L);
- manager.triggerWatch(path2, EventType.NodeDeleted);
+ manager.triggerWatch(path2, EventType.NodeDeleted, null);
checkMetrics("node_deleted_watch_count", 1L, 1L, 1D, 1L, 1L);
//make sure that node created watch count is not impacted by the fire of other event types
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ACLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ACLTest.java
index d1b5a1065e4..5ffd69e86a1 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ACLTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ACLTest.java
@@ -19,7 +19,10 @@
package org.apache.zookeeper.test;
import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
+import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import java.io.File;
@@ -27,16 +30,19 @@
import java.util.List;
import java.util.concurrent.CountDownLatch;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.InvalidACLException;
import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZKTestCase;
+import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.server.ServerCnxn;
import org.apache.zookeeper.server.ServerCnxnFactory;
import org.apache.zookeeper.server.SyncRequestProcessor;
@@ -302,4 +308,91 @@ public void testNullValueACL() throws Exception {
}
}
+ @Test
+ public void testExistACLCheck() throws Exception {
+ File tmpDir = ClientBase.createTmpDir();
+ ClientBase.setupTestEnv();
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
+ ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
+ f.startup(zks);
+ String path = "/testExistACLCheck";
+ String data = "/testExistACLCheck-data";
+ try {
+ LOG.info("starting up the zookeeper server .. waiting");
+ assertTrue(ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT), "waiting for server being up");
+ ZooKeeper zk = ClientBase.createZKClient(HOSTPORT);
+ try {
+ Stat stat = zk.exists(path, false);
+ assertNull(stat);
+ zk.create(path, data.getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ stat = zk.exists(path, false);
+ assertNotNull(stat);
+ assertEquals(data.length(), stat.getDataLength());
+
+ zk.delete(path, -1);
+ ArrayList acls = new ArrayList<>();
+ acls.add(new ACL(ZooDefs.Perms.WRITE, Ids.ANYONE_ID_UNSAFE));
+ zk.create(path, data.getBytes(), acls, CreateMode.PERSISTENT);
+ try {
+ stat = zk.exists(path, false);
+ fail("exists should throw NoAuthException when don't have read permission");
+ } catch (KeeperException.NoAuthException e) {
+ //expected
+ }
+
+ zk.delete(path, -1);
+ acls = new ArrayList<>();
+ acls.add(new ACL(ZooDefs.Perms.READ, Ids.ANYONE_ID_UNSAFE));
+ zk.create(path, data.getBytes(), acls, CreateMode.PERSISTENT);
+ stat = zk.exists(path, false);
+ assertNotNull(stat);
+ assertEquals(data.length(), stat.getDataLength());
+ } finally {
+ zk.close();
+ }
+ } finally {
+ f.shutdown();
+ zks.shutdown();
+ assertTrue(ClientBase.waitForServerDown(HOSTPORT, ClientBase.CONNECTION_TIMEOUT), "waiting for server down");
+ }
+ }
+
+ @Test
+ public void testExistACLCheckAtRootPath() throws Exception {
+ File tmpDir = ClientBase.createTmpDir();
+ ClientBase.setupTestEnv();
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
+ ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
+ f.startup(zks);
+ try {
+ LOG.info("starting up the zookeeper server .. waiting");
+ assertTrue(ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT), "waiting for server being up");
+ ZooKeeper zk = ClientBase.createZKClient(HOSTPORT);
+ try {
+ String data = "/testExistACLCheckAtRootPath-data";
+ zk.create("/a", data.getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ ArrayList acls = new ArrayList<>();
+ acls.add(new ACL(0, Ids.ANYONE_ID_UNSAFE));
+ zk.setACL("/", acls, -1);
+
+ Stat stat = zk.exists("/a", false);
+ assertNotNull(stat);
+ assertEquals(data.length(), stat.getDataLength());
+ try {
+ stat = zk.exists("/", false);
+ fail("exists should throw NoAuthException when removing root path's ACL permission");
+ } catch (KeeperException.NoAuthException e) {
+ //expected
+ }
+ } finally {
+ zk.close();
+ }
+ } finally {
+ f.shutdown();
+ zks.shutdown();
+ assertTrue(ClientBase.waitForServerDown(HOSTPORT, ClientBase.CONNECTION_TIMEOUT), "waiting for server down");
+ }
+ }
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java
index 0b156156d0d..fca48c751f1 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java
@@ -25,12 +25,15 @@
import static org.junit.jupiter.api.Assertions.fail;
import java.io.IOException;
import java.util.Arrays;
+import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooKeeper;
import org.junit.jupiter.api.Test;
@@ -60,6 +63,25 @@ public boolean matches() throws InterruptedException {
}
+ @Test
+ public void testChrootWithZooKeeperPathWatcher() throws Exception {
+ ZooKeeper zk1 = createClient(hostPort + "/chroot");
+ BlockingQueue events = new LinkedBlockingQueue<>();
+ byte[] config = zk1.getConfig(events::add, null);
+
+ ZooKeeper zk2 = createClient();
+ zk2.addAuthInfo("digest", "super:test".getBytes());
+ zk2.setData(ZooDefs.CONFIG_NODE, config, -1);
+
+ waitFor("config watcher receive no event", () -> !events.isEmpty(), 10);
+
+ WatchedEvent event = events.poll();
+ assertNotNull(event);
+ assertEquals(Watcher.Event.KeeperState.SyncConnected, event.getState());
+ assertEquals(Watcher.Event.EventType.NodeDataChanged, event.getType());
+ assertEquals(ZooDefs.CONFIG_NODE, event.getPath());
+ }
+
@Test
public void testChrootSynchronous() throws IOException, InterruptedException, KeeperException {
ZooKeeper zk1 = createClient();
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java
index 43878e0b030..392c636dea8 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java
@@ -22,11 +22,13 @@
package org.apache.zookeeper.test;
+import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
import static org.hamcrest.CoreMatchers.startsWith;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.io.IOException;
+import java.net.InetAddress;
import java.nio.file.Path;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.PortAssignment;
@@ -42,6 +44,9 @@
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
public class ClientSSLTest extends QuorumPeerTestBase {
@@ -73,6 +78,8 @@ public void teardown() {
System.clearProperty(clientX509Util.getSslTruststoreLocationProperty());
System.clearProperty(clientX509Util.getSslTruststorePasswdProperty());
System.clearProperty(clientX509Util.getSslTruststorePasswdPathProperty());
+ System.clearProperty(clientX509Util.getFipsModeProperty());
+ System.clearProperty(clientX509Util.getSslHostnameVerificationEnabledProperty());
clientX509Util.close();
}
@@ -106,12 +113,36 @@ public void testClientServerSSLWithCnxnClassName() throws Exception {
* 2. setting jvm flags for serverCnxn, keystore, truststore.
* Finally, a zookeeper client should be able to connect to the secure port and
* communicate with server via secure connection.
- *
+ *
* Note that in this test a ZK server has two ports -- clientPort and secureClientPort.
+ *
+ * This test covers the positive scenarios for hostname verification.
*/
- @Test
- public void testClientServerSSL() throws Exception {
- testClientServerSSL(true);
+ @ParameterizedTest(name = "fipsEnabled={0}, hostnameVerification={1}")
+ @CsvSource({"true,true", "true,false", "false,true", "false,false"})
+ public void testClientServerSSL_positive(String fipsEnabled, String hostnameVerification) throws Exception {
+ // Arrange
+ System.setProperty(clientX509Util.getFipsModeProperty(), fipsEnabled);
+ System.setProperty(clientX509Util.getSslHostnameVerificationEnabledProperty(), hostnameVerification);
+
+ // Act & Assert
+ testClientServerSSL(hostnameVerification.equals("true") ? "localhost" : InetAddress.getLocalHost().getHostName(),
+ true, CONNECTION_TIMEOUT);
+ }
+
+ /**
+ * This test covers the negative scenarios for hostname verification.
+ */
+ @ParameterizedTest(name = "fipsEnabled={0}")
+ @ValueSource(booleans = { true, false })
+ public void testClientServerSSL_negative(boolean fipsEnabled) {
+ // Arrange
+ System.setProperty(clientX509Util.getFipsModeProperty(), Boolean.toString(fipsEnabled));
+ System.setProperty(clientX509Util.getSslHostnameVerificationEnabledProperty(), "true");
+
+ // Act & Assert
+ assertThrows(AssertionError.class, () ->
+ testClientServerSSL(InetAddress.getLocalHost().getHostName(), true, 5000));
}
@Test
@@ -128,6 +159,10 @@ public void testClientServerSSL_withPasswordFromFile() throws Exception {
}
public void testClientServerSSL(boolean useSecurePort) throws Exception {
+ testClientServerSSL("localhost", useSecurePort, CONNECTION_TIMEOUT);
+ }
+
+ public void testClientServerSSL(String hostname, boolean useSecurePort, long connectTimeout) throws Exception {
final int SERVER_COUNT = 3;
final int[] clientPorts = new int[SERVER_COUNT];
final Integer[] secureClientPorts = new Integer[SERVER_COUNT];
@@ -159,11 +194,11 @@ public void testClientServerSSL(boolean useSecurePort) throws Exception {
assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], TIMEOUT),
"waiting for server " + i + " being up");
final int port = useSecurePort ? secureClientPorts[i] : clientPorts[i];
- ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + port, TIMEOUT);
- // Do a simple operation to make sure the connection is fine.
- zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- zk.delete("/test", -1);
- zk.close();
+ try (ZooKeeper zk = ClientBase.createZKClient(hostname + ":" + port, TIMEOUT, connectTimeout)) {
+ // Do a simple operation to make sure the connection is fine.
+ zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.delete("/test", -1);
+ }
}
for (int i = 0; i < mt.length; i++) {
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLETest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLETest.java
index 15713e423cc..0f679b2d268 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLETest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLETest.java
@@ -368,7 +368,7 @@ public VerifyState(QuorumPeer peer) {
this.peer = peer;
}
public void run() {
- setName("VerifyState-" + peer.getId());
+ setName("VerifyState-" + peer.getMyId());
while (true) {
if (peer.getPeerState() == ServerState.FOLLOWING) {
LOG.info("I am following");
@@ -410,7 +410,7 @@ public void testJoin() throws Exception {
// start 2 peers and verify if they form the cluster
for (sid = 0; sid < 2; sid++) {
peer = new QuorumPeer(peers, tmpdir[sid], tmpdir[sid], port[sid], 3, sid, 2000, 2, 2, 2);
- LOG.info("Starting peer {}", peer.getId());
+ LOG.info("Starting peer {}", peer.getMyId());
peer.start();
peerList.add(sid, peer);
}
@@ -421,16 +421,16 @@ public void testJoin() throws Exception {
assertFalse(!v1.isSuccess(), "Unable to form cluster in " + waitTime + " ms");
// Start 3rd peer and check if it goes in LEADING state
peer = new QuorumPeer(peers, tmpdir[sid], tmpdir[sid], port[sid], 3, sid, 2000, 2, 2, 2);
- LOG.info("Starting peer {}", peer.getId());
+ LOG.info("Starting peer {}", peer.getMyId());
peer.start();
peerList.add(sid, peer);
v1 = new VerifyState(peer);
v1.start();
v1.join(waitTime);
if (v1.isAlive()) {
- fail("Peer " + peer.getId() + " failed to join the cluster " + "within " + waitTime + " ms");
+ fail("Peer " + peer.getMyId() + " failed to join the cluster " + "within " + waitTime + " ms");
} else if (!v1.isSuccess()) {
- fail("Incorrect LEADING state for peer " + peer.getId());
+ fail("Incorrect LEADING state for peer " + peer.getMyId());
}
// cleanup
for (int id = 0; id < 3; id++) {
@@ -459,7 +459,7 @@ public void testJoinInconsistentEnsemble() throws Exception {
// start 2 peers and verify if they form the cluster
for (sid = 0; sid < 2; sid++) {
peer = new QuorumPeer(peers, tmpdir[sid], tmpdir[sid], port[sid], 3, sid, 2000, 2, 2, 2);
- LOG.info("Starting peer {}", peer.getId());
+ LOG.info("Starting peer {}", peer.getMyId());
peer.start();
peerList.add(sid, peer);
}
@@ -478,14 +478,14 @@ public void testJoinInconsistentEnsemble() throws Exception {
peer.setCurrentVote(newVote);
// Start 3rd peer and check if it joins the quorum
peer = new QuorumPeer(peers, tmpdir[2], tmpdir[2], port[2], 3, 2, 2000, 2, 2, 2);
- LOG.info("Starting peer {}", peer.getId());
+ LOG.info("Starting peer {}", peer.getMyId());
peer.start();
peerList.add(sid, peer);
v1 = new VerifyState(peer);
v1.start();
v1.join(waitTime);
if (v1.isAlive()) {
- fail("Peer " + peer.getId() + " failed to join the cluster " + "within " + waitTime + " ms");
+ fail("Peer " + peer.getMyId() + " failed to join the cluster " + "within " + waitTime + " ms");
}
// cleanup
for (int id = 0; id < 3; id++) {
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLEZeroWeightTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLEZeroWeightTest.java
index 0fdac4556a3..59ed266f6d7 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLEZeroWeightTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLEZeroWeightTest.java
@@ -128,7 +128,7 @@ public void run() {
LOG.info("Finished election: {}, {}", i, v.getId());
votes[i] = v;
- if ((peer.getPeerState() == ServerState.LEADING) && (peer.getId() > 2)) {
+ if ((peer.getPeerState() == ServerState.LEADING) && (peer.getMyId() > 2)) {
fail = true;
}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiOperationTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiOperationTest.java
index 4ff593222fe..53c6562532a 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiOperationTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiOperationTest.java
@@ -21,6 +21,7 @@
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertSame;
@@ -40,6 +41,7 @@
import org.apache.zookeeper.AsyncCallback.MultiCallback;
import org.apache.zookeeper.ClientCnxn;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.CreateOptions;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
@@ -443,6 +445,28 @@ public void testCreate(boolean useAsync) throws Exception {
zk.getData("/multi2", false, null);
}
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void testCreate2(boolean useAsync) throws Exception {
+ CreateOptions options = CreateOptions.newBuilder(Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT).build();
+ List ops = Arrays.asList(
+ Op.create("/multi0", new byte[0], options),
+ Op.create("/multi1", new byte[0], options),
+ Op.create("/multi2", new byte[0], options));
+ List results = multi(zk, ops, useAsync);
+ for (int i = 0; i < ops.size(); i++) {
+ CreateResult createResult = (CreateResult) results.get(i);
+ assertEquals(ops.get(i).getPath(), createResult.getPath());
+ assertEquals(ZooDefs.OpCode.create2, createResult.getType(), createResult.getPath());
+ assertNotNull(createResult.getStat(), createResult.getPath());
+ assertNotEquals(0, createResult.getStat().getCzxid(), createResult.getPath());
+ }
+
+ zk.getData("/multi0", false, null);
+ zk.getData("/multi1", false, null);
+ zk.getData("/multi2", false, null);
+ }
+
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testEmpty(boolean useAsync) throws Exception {
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/NonRecoverableErrorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/NonRecoverableErrorTest.java
index 3eaa1d3ffcf..ed592df6f7c 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/NonRecoverableErrorTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/NonRecoverableErrorTest.java
@@ -93,7 +93,7 @@ public void testZooKeeperServiceAvailableOnLeader() throws Exception {
// inject problem in leader
FileTxnSnapLog snapLog = leader.getActiveServer().getTxnLogFactory();
- FileTxnSnapLog fileTxnSnapLogWithError = new FileTxnSnapLog(snapLog.getDataDir(), snapLog.getSnapDir()) {
+ FileTxnSnapLog fileTxnSnapLogWithError = new FileTxnSnapLog(snapLog.getDataLogDir(), snapLog.getSnapDir()) {
@Override
public void commit() throws IOException {
throw new IOException("Input/output error");
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java
index 4ba33c7a854..ffd65997866 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java
@@ -330,7 +330,7 @@ public void testAdminCommands(boolean testObserverMaster) throws IOException, MB
ObjectName connBean = null;
for (ObjectName bean : JMXEnv.conn().queryNames(new ObjectName(MBeanRegistry.DOMAIN + ":*"), null)) {
if (bean.getCanonicalName().contains("Learner_Connections") && bean.getCanonicalName().contains("id:"
- + q3.getQuorumPeer().getId())) {
+ + q3.getQuorumPeer().getMyId())) {
connBean = bean;
break;
}
@@ -344,7 +344,7 @@ public void testAdminCommands(boolean testObserverMaster) throws IOException, MB
assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_OBS, CONNECTION_TIMEOUT),
"waiting for server 3 being up");
- final String obsBeanName = String.format("org.apache.ZooKeeperService:name0=ReplicatedServer_id%d,name1=replica.%d,name2=Observer", q3.getQuorumPeer().getId(), q3.getQuorumPeer().getId());
+ final String obsBeanName = String.format("org.apache.ZooKeeperService:name0=ReplicatedServer_id%d,name1=replica.%d,name2=Observer", q3.getQuorumPeer().getMyId(), q3.getQuorumPeer().getMyId());
Set names = JMXEnv.conn().queryNames(new ObjectName(obsBeanName), null);
assertEquals(1, names.size(), "expecting singular observer bean");
ObjectName obsBean = names.iterator().next();
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherACLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherACLTest.java
new file mode 100644
index 00000000000..1597a4850e9
--- /dev/null
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherACLTest.java
@@ -0,0 +1,629 @@
+/**
+ * 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.zookeeper.test;
+
+import static org.apache.zookeeper.AddWatchMode.PERSISTENT;
+import static org.apache.zookeeper.AddWatchMode.PERSISTENT_RECURSIVE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.fail;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class encodes a set of tests corresponding to a "truth table"
+ * of interactions between persistent watchers and znode ACLs:
+ *
+ * https://docs.google.com/spreadsheets/d/1eMH2aimrrMc_b6McU8CHm2yCj2X-w30Fy4fCBOHn7NA/edit#gid=0
+ */
+public class PersistentWatcherACLTest extends ClientBase {
+ private static final Logger LOG = LoggerFactory.getLogger(PersistentWatcherACLTest.class);
+ /** An ACL denying READ. */
+ private static final List ACL_NO_READ = Collections.singletonList(new ACL(ZooDefs.Perms.ALL & ~ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE));
+ private BlockingQueue events;
+ private Watcher persistentWatcher;
+
+ @Override
+ @BeforeEach
+ public void setUp() throws Exception {
+ super.setUp();
+
+ events = new LinkedBlockingQueue<>();
+ persistentWatcher = event -> {
+ events.add(event);
+ LOG.info("Added event: {}; total: {}", event, events.size());
+ };
+ }
+
+ /**
+ * This Step class, with the Round class below, is used to encode
+ * the contents of the truth table.
+ *
+ * (These should become Records once we target JDK 14+.)
+ */
+ private static class Step {
+ Step(int opCode, String target) {
+ this(opCode, target, null, null);
+ }
+ Step(int opCode, String target, EventType eventType, String eventPath) {
+ this.opCode = opCode;
+ this.target = target;
+ this.eventType = eventType;
+ this.eventPath = eventPath;
+ }
+ /** Action: create, setData or delete */
+ final int opCode;
+ /** Target path */
+ final String target;
+ /** Expected event type, {@code null} if no event is expected */
+ final EventType eventType;
+ /** Expected event path, {@code null} if no event is expected */
+ final String eventPath;
+ }
+
+ /**
+ * This Round class, with the Step class above, is used to encode
+ * the contents of the truth table.
+ *
+ * (These should become Records once we target JDK 14+.)
+ */
+ private static class Round {
+ Round(String summary, Boolean allowA, Boolean allowB, Boolean allowC, String watchTarget, AddWatchMode watchMode, Step[] steps) {
+ this.summary = summary;
+ this.allowA = allowA;
+ this.allowB = allowB;
+ this.allowC = allowC;
+ this.watchTarget = watchTarget;
+ this.watchMode = watchMode;
+ this.steps = steps;
+ }
+ /** Notes/summary */
+ final String summary;
+ /** Should /a's ACL leave it readable? */
+ final Boolean allowA;
+ /** Should /a/b's ACL leave it readable? */
+ final Boolean allowB;
+ /** Should /a/b/c's ACL leave it readable? */
+ final Boolean allowC;
+ /** Watch path */
+ final String watchTarget;
+ /** Watch mode */
+ final AddWatchMode watchMode;
+ /** Actions and expected events */
+ final Step[] steps;
+ }
+
+ /**
+ * A "round" of tests from the table encoded as Java objects.
+ *
+ * Note that the set of rounds is collected in a {@code ROUNDS}
+ * array below, and that this test class includes a {@code main}
+ * method which produces a "CSV" rendition of the table, for ease
+ * of comparison with the original.
+ *
+ * @see #ROUNDS
+ */
+ private static final Round roundNothingAsAIsWatchedButDeniedBIsNotWatched =
+ new Round(
+ "Nothing as a is watched but denied. b is not watched",
+ false, true, null, "/a", PERSISTENT, new Step[] {
+ new Step(ZooDefs.OpCode.setData, "/a"),
+ new Step(ZooDefs.OpCode.create, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundNothingAsBothAAndBDenied =
+ new Round(
+ "Nothing as both a and b denied",
+ false, false, null, "/a", PERSISTENT, new Step[] {
+ new Step(ZooDefs.OpCode.setData, "/a"),
+ new Step(ZooDefs.OpCode.create, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundAChangesInclChildrenAreSeen =
+ new Round(
+ "a changes, incl children, are seen",
+ true, false, null, "/a", PERSISTENT, new Step[] {
+ new Step(ZooDefs.OpCode.create, "/a", EventType.NodeCreated, "/a"),
+ new Step(ZooDefs.OpCode.setData, "/a", EventType.NodeDataChanged, "/a"),
+ new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeChildrenChanged, "/a"),
+ new Step(ZooDefs.OpCode.setData, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeChildrenChanged, "/a"),
+ new Step(ZooDefs.OpCode.delete, "/a", EventType.NodeDeleted, "/a"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundNothingForAAsItSDeniedBChangesSeen =
+ new Round(
+ "Nothing for a as it's denied, b changes allowed/seen",
+ false, true, null, "/a", PERSISTENT_RECURSIVE, new Step[] {
+ new Step(ZooDefs.OpCode.setData, "/a"),
+ new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeCreated, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b", EventType.NodeDataChanged, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeDeleted, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundNothingBothDenied =
+ new Round(
+ "Nothing - both denied",
+ false, false, null, "/a", PERSISTENT_RECURSIVE, new Step[] {
+ new Step(ZooDefs.OpCode.setData, "/a"),
+ new Step(ZooDefs.OpCode.create, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a/b"),
+ new Step(ZooDefs.OpCode.delete, "/a"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundNothingAllDenied =
+ new Round(
+ "Nothing - all denied",
+ false, false, false, "/a", PERSISTENT_RECURSIVE, new Step[] {
+ new Step(ZooDefs.OpCode.create, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b"),
+ new Step(ZooDefs.OpCode.create, "/a/b/c"),
+ new Step(ZooDefs.OpCode.setData, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundADeniesSeeAllChangesForBAndCIncludingBChildren =
+ new Round(
+ "a denies, see all changes for b and c, including b's children",
+ false, true, true, "/a", PERSISTENT_RECURSIVE, new Step[] {
+ new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeCreated, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b", EventType.NodeDataChanged, "/a/b"),
+ new Step(ZooDefs.OpCode.create, "/a/b/c", EventType.NodeCreated, "/a/b/c"),
+ new Step(ZooDefs.OpCode.setData, "/a/b/c", EventType.NodeDataChanged, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b/c", EventType.NodeDeleted, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeDeleted, "/a/b"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundADeniesSeeAllBChangesAndBChildrenNothingForC =
+ new Round(
+ "a denies, see all b changes and b's children, nothing for c",
+ false, true, false, "/a", PERSISTENT_RECURSIVE, new Step[] {
+ new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeCreated, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b", EventType.NodeDataChanged, "/a/b"),
+ new Step(ZooDefs.OpCode.create, "/a/b/c"),
+ new Step(ZooDefs.OpCode.setData, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeDeleted, "/a/b"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundNothingTheWatchIsOnC =
+ new Round(
+ "Nothing - the watch is on c",
+ false, true, false, "/a/b/c", PERSISTENT_RECURSIVE, new Step[] {
+ new Step(ZooDefs.OpCode.create, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b"),
+ new Step(ZooDefs.OpCode.create, "/a/b/c"),
+ new Step(ZooDefs.OpCode.setData, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b"),
+ }
+ );
+
+ /**
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round roundTheWatchIsOnlyOnCBAndCAllowed =
+ new Round(
+ "The watch is only on c (b and c allowed)",
+ false, true, true, "/a/b/c", PERSISTENT_RECURSIVE, new Step[] {
+ new Step(ZooDefs.OpCode.create, "/a/b"),
+ new Step(ZooDefs.OpCode.setData, "/a/b"),
+ new Step(ZooDefs.OpCode.create, "/a/b/c", EventType.NodeCreated, "/a/b/c"),
+ new Step(ZooDefs.OpCode.setData, "/a/b/c", EventType.NodeDataChanged, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b/c", EventType.NodeDeleted, "/a/b/c"),
+ new Step(ZooDefs.OpCode.delete, "/a/b"),
+ }
+ );
+
+ /**
+ * Transform the "tristate" {@code allow} property to a concrete
+ * ACL which can be passed to the ZooKeeper API.
+ *
+ * @param allow "tristate" value: {@code null}/don't care, {@code
+ * true}, {@code false}
+ * @return the ACL
+ */
+ private static List selectAcl(Boolean allow) {
+ if (allow == null) {
+ return null;
+ } else if (!allow) {
+ return ACL_NO_READ;
+ } else {
+ return ZooDefs.Ids.OPEN_ACL_UNSAFE;
+ }
+ }
+
+ /**
+ * Executes one "round" of tests from the Java object encoding of
+ * the table.
+ *
+ * @param round the "round"
+ *
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see PersistentWatcherACLTest.Round
+ * @see PersistentWatcherACLTest.Step
+ */
+ private void execRound(Round round)
+ throws IOException, InterruptedException, KeeperException {
+ try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) {
+ List aclForA = selectAcl(round.allowA);
+ List aclForB = selectAcl(round.allowB);
+ List aclForC = selectAcl(round.allowC);
+
+ boolean firstStepCreatesA = round.steps.length > 0
+ && round.steps[0].opCode == ZooDefs.OpCode.create
+ && round.steps[0].target.equals("/a");
+
+ // Assume /a always exists (except if it's about to be created)
+ if (!firstStepCreatesA) {
+ zk.create("/a", new byte[0], aclForA, CreateMode.PERSISTENT);
+ }
+
+ zk.addWatch(round.watchTarget, persistentWatcher, round.watchMode);
+
+ for (int i = 0; i < round.steps.length; i++) {
+ Step step = round.steps[i];
+
+ switch (step.opCode) {
+ case ZooDefs.OpCode.create:
+ List acl = step.target.endsWith("/c")
+ ? aclForC
+ : step.target.endsWith("/b")
+ ? aclForB
+ : aclForA;
+ zk.create(step.target, new byte[0], acl, CreateMode.PERSISTENT);
+ break;
+ case ZooDefs.OpCode.delete:
+ zk.delete(step.target, -1);
+ break;
+ case ZooDefs.OpCode.setData:
+ zk.setData(step.target, new byte[0], -1);
+ break;
+ default:
+ fail("Unexpected opCode " + step.opCode + " in step " + i);
+ break;
+ }
+
+ WatchedEvent actualEvent = events.poll(500, TimeUnit.MILLISECONDS);
+ if (step.eventType == null) {
+ assertNull(actualEvent, "Unexpected event " + actualEvent + " at step " + i);
+ } else {
+ String m = "In event " + actualEvent + " at step " + i;
+ assertNotNull(actualEvent, m);
+ assertEquals(step.eventType, actualEvent.getType(), m);
+ assertEquals(step.eventPath, actualEvent.getPath(), m);
+ }
+ }
+ }
+ }
+
+ /**
+ * A test method, wrapping the definition of a "round." This
+ * should really use JUnit 5's runtime test case generation
+ * facilities, but that would prevent backporting this suite to
+ * JUnit 4.
+ *
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see JUnit 5 runtime test case generation
+ */
+ @Test
+ public void testNothingAsAIsWatchedButDeniedBIsNotWatched()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundNothingAsAIsWatchedButDeniedBIsNotWatched);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundNothingAsBothAAndBDenied
+ */
+ @Test
+ public void testNothingAsBothAAndBDenied()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundNothingAsBothAAndBDenied);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundAChangesInclChildrenAreSeen
+ */
+ @Test
+ public void testAChangesInclChildrenAreSeen()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundAChangesInclChildrenAreSeen);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundNothingForAAsItSDeniedBChangesSeen
+ */
+ @Test
+ public void testNothingForAAsItSDeniedBChangesSeen()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundNothingForAAsItSDeniedBChangesSeen);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundNothingBothDenied
+ */
+ @Test
+ public void testNothingBothDenied()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundNothingBothDenied);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundNothingAllDenied
+ */
+ @Test
+ public void testNothingAllDenied()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundNothingAllDenied);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundADeniesSeeAllChangesForBAndCIncludingBChildren
+ */
+ @Test
+ public void testADeniesSeeAllChangesForBAndCIncludingBChildren()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundADeniesSeeAllChangesForBAndCIncludingBChildren);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundADeniesSeeAllBChangesAndBChildrenNothingForC
+ */
+ @Test
+ public void testADeniesSeeAllBChangesAndBChildrenNothingForC()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundADeniesSeeAllBChangesAndBChildrenNothingForC);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundNothingTheWatchIsOnC
+ */
+ @Test
+ public void testNothingTheWatchIsOnC()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundNothingTheWatchIsOnC);
+ }
+
+ /**
+ * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+ * @see #roundTheWatchIsOnlyOnCBAndCAllowed
+ */
+ @Test
+ public void testTheWatchIsOnlyOnCBAndCAllowed()
+ throws IOException, InterruptedException, KeeperException {
+ execRound(roundTheWatchIsOnlyOnCBAndCAllowed);
+ }
+
+ // The rest of this class is the world's lamest "CSV" encoder.
+
+ /**
+ * The set of rounds. This array includes one entry for each
+ * {@code private static final Round round*} member variable
+ * defined above.
+ *
+ * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+ */
+ private static final Round[] ROUNDS = new Round[] {
+ roundNothingAsAIsWatchedButDeniedBIsNotWatched,
+ roundNothingAsBothAAndBDenied,
+ roundAChangesInclChildrenAreSeen,
+ roundNothingForAAsItSDeniedBChangesSeen,
+ roundNothingBothDenied,
+ roundNothingAllDenied,
+ roundADeniesSeeAllChangesForBAndCIncludingBChildren,
+ roundADeniesSeeAllBChangesAndBChildrenNothingForC,
+ roundNothingTheWatchIsOnC,
+ roundTheWatchIsOnlyOnCBAndCAllowed,
+ };
+
+ private static String allowString(String prefix, Boolean allow) {
+ if (allow == null) {
+ return "";
+ } else {
+ return prefix + (allow ? "allow" : "deny");
+ }
+ }
+
+ private static String watchModeString(AddWatchMode watchMode) {
+ switch (watchMode) {
+ case PERSISTENT:
+ return "PERSISTENT";
+ case PERSISTENT_RECURSIVE:
+ return "PRECURSIVE";
+ default:
+ return "?";
+ }
+ }
+
+ private static String actionString(int opCode) {
+ switch (opCode) {
+ case ZooDefs.OpCode.create:
+ return "create";
+ case ZooDefs.OpCode.delete:
+ return "delete";
+ case ZooDefs.OpCode.setData:
+ return "modify";
+ default:
+ return "?";
+ }
+ }
+
+ private static String eventPathString(String eventPath) {
+ if (eventPath == null) {
+ return "?";
+ } else if (eventPath.length() <= 1) {
+ return eventPath;
+ } else {
+ return eventPath.substring(eventPath.lastIndexOf('/') + 1);
+ }
+ }
+
+ /**
+ * Generates a "CSV" rendition of the table in sb.
+ *
+ * @param sb the target string builder
+ */
+ private static void genCsv(StringBuilder sb) {
+ sb.append("Initial State,")
+ .append("Action,")
+ .append("NodeCreated,")
+ .append("NodeDeleted,")
+ .append("NodeDataChanged,")
+ .append("NodeChildrenChanged,")
+ .append("Notes/summary\n");
+ sb.append("Assume /a always exists\n\n");
+
+ for (Round round : ROUNDS) {
+ sb.append("\"ACL")
+ .append(allowString(": a ", round.allowA))
+ .append(allowString(", b ", round.allowB))
+ .append(allowString(", c ", round.allowC))
+ .append("\"")
+ .append(",,,,,,\"")
+ .append(round.summary)
+ .append("\"\n");
+ for (int i = 0; i < round.steps.length; i++) {
+ Step step = round.steps[i];
+
+ if (i == 0) {
+ sb.append("\"addWatch(")
+ .append(round.watchTarget)
+ .append(", ")
+ .append(watchModeString(round.watchMode))
+ .append(")\"");
+ }
+
+ sb.append(",")
+ .append(actionString(step.opCode))
+ .append(" ")
+ .append(step.target)
+ .append(",");
+
+ if (step.eventType == EventType.NodeCreated) {
+ sb.append("y - ")
+ .append(eventPathString(step.eventPath));
+ }
+
+ sb.append(",");
+
+ if (step.eventType == EventType.NodeDeleted) {
+ sb.append("y - ")
+ .append(eventPathString(step.eventPath));
+ }
+
+ sb.append(",");
+
+ if (step.eventType == EventType.NodeDataChanged) {
+ sb.append("y - ")
+ .append(eventPathString(step.eventPath));
+ }
+
+ sb.append(",");
+
+ if (round.watchMode == PERSISTENT_RECURSIVE) {
+ sb.append("n");
+ } else if (step.eventType == EventType.NodeChildrenChanged) {
+ sb.append("y - ")
+ .append(eventPathString(step.eventPath));
+ }
+
+ sb.append("\n");
+ }
+
+ sb.append("\n");
+ }
+ }
+
+ /**
+ * Generates a "CSV" rendition of the table to standard output.
+ *
+ * @see #ROUNDS
+ */
+ public static void main(String[] args) {
+ StringBuilder sb = new StringBuilder();
+ genCsv(sb);
+ System.out.println(sb);
+ }
+}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java
index 8f5f17db7fd..4f723f2994f 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java
@@ -47,7 +47,7 @@ public class QuorumBase extends ClientBase {
private static final Logger LOG = LoggerFactory.getLogger(QuorumBase.class);
- private static final String LOCALADDR = "127.0.0.1";
+ protected static final String LOCALADDR = "127.0.0.1";
private File oracleDir;
private static final String oraclePath_0 = "/oraclePath/0/mastership/";
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumTest.java
index 760d9a4d7c1..72a5f140b52 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumTest.java
@@ -115,10 +115,10 @@ public void testGetView() {
@Test
public void testViewContains() {
// Test view contains self
- assertTrue(qb.s1.viewContains(qb.s1.getId()));
+ assertTrue(qb.s1.viewContains(qb.s1.getMyId()));
// Test view contains other servers
- assertTrue(qb.s1.viewContains(qb.s2.getId()));
+ assertTrue(qb.s1.viewContains(qb.s2.getMyId()));
// Test view does not contain non-existant servers
assertFalse(qb.s1.viewContains(-1L));
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/RepeatStartupTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/RepeatStartupTest.java
index 4dff8e27621..8c79af34104 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/RepeatStartupTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/RepeatStartupTest.java
@@ -49,7 +49,7 @@ public void testFail() throws Exception {
QuorumBase.shutdown(qb.s4);
QuorumBase.shutdown(qb.s5);
String hp = qb.hostPort.split(",")[0];
- ZooKeeperServer zks = new ZooKeeperServer(qb.s1.getTxnFactory().getSnapDir(), qb.s1.getTxnFactory().getDataDir(), 3000);
+ ZooKeeperServer zks = new ZooKeeperServer(qb.s1.getTxnFactory().getSnapDir(), qb.s1.getTxnFactory().getDataLogDir(), 3000);
final int PORT = Integer.parseInt(hp.split(":")[1]);
ServerCnxnFactory factory = ServerCnxnFactory.createFactory(PORT, -1);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/StaticHostProviderTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/StaticHostProviderTest.java
index 905b6cd78c3..70460606af3 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/StaticHostProviderTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/StaticHostProviderTest.java
@@ -30,11 +30,9 @@
import static org.junit.jupiter.api.Assertions.assertSame;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
@@ -42,16 +40,42 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
+import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Random;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.client.HostProvider;
import org.apache.zookeeper.client.StaticHostProvider;
import org.apache.zookeeper.common.Time;
+import org.burningwave.tools.net.DefaultHostResolver;
+import org.burningwave.tools.net.HostResolutionRequestInterceptor;
+import org.burningwave.tools.net.MappedHostResolver;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
public class StaticHostProviderTest extends ZKTestCase {
+ @BeforeAll
+ public static void setupDNSMocks() {
+ Map hostAliases = new LinkedHashMap<>();
+ hostAliases.put("site1.mock", "192.168.1.1");
+ hostAliases.put("site2.mock", "192.168.1.2");
+ hostAliases.put("site3.mock", "192.168.1.3");
+ hostAliases.put("site4.mock", "192.168.1.4");
+
+ HostResolutionRequestInterceptor.INSTANCE.install(
+ new MappedHostResolver(hostAliases),
+ DefaultHostResolver.INSTANCE
+ );
+ }
+
+ @AfterAll
+ public static void clearDNSMocks() {
+ HostResolutionRequestInterceptor.INSTANCE.uninstall();
+ }
+
private Random r = new Random(1);
@Test
@@ -704,10 +728,7 @@ public void testReResolvingSingle() throws UnknownHostException {
// Test a hostname that resolves to a single address
list.add(InetSocketAddress.createUnresolved("issues.apache.org", 1234));
- final InetAddress issuesApacheOrg = mock(InetAddress.class);
- when(issuesApacheOrg.getHostAddress()).thenReturn("192.168.1.1");
- when(issuesApacheOrg.toString()).thenReturn("issues.apache.org");
- when(issuesApacheOrg.getHostName()).thenReturn("issues.apache.org");
+ final InetAddress issuesApacheOrg = InetAddress.getByName("site1.mock");
StaticHostProvider.Resolver resolver = new StaticHostProvider.Resolver() {
@Override
@@ -738,15 +759,9 @@ public void testReResolvingMultiple() throws UnknownHostException {
// Test a hostname that resolves to multiple addresses
list.add(InetSocketAddress.createUnresolved("www.apache.org", 1234));
- final InetAddress apacheOrg1 = mock(InetAddress.class);
- when(apacheOrg1.getHostAddress()).thenReturn("192.168.1.1");
- when(apacheOrg1.toString()).thenReturn("www.apache.org");
- when(apacheOrg1.getHostName()).thenReturn("www.apache.org");
+ final InetAddress apacheOrg1 = InetAddress.getByName("site1.mock");
- final InetAddress apacheOrg2 = mock(InetAddress.class);
- when(apacheOrg2.getHostAddress()).thenReturn("192.168.1.2");
- when(apacheOrg2.toString()).thenReturn("www.apache.org");
- when(apacheOrg2.getHostName()).thenReturn("www.apache.org");
+ final InetAddress apacheOrg2 = InetAddress.getByName("site2.mock");
final List resolvedAddresses = new ArrayList();
resolvedAddresses.add(apacheOrg1);
@@ -781,10 +796,7 @@ public void testReResolveMultipleOneFailing() throws UnknownHostException {
final List resolvedAddresses = new ArrayList();
for (int i = 0; i < 3; i++) {
ipList.add(String.format("192.168.1.%d", i + 1));
- final InetAddress apacheOrg = mock(InetAddress.class);
- when(apacheOrg.getHostAddress()).thenReturn(String.format("192.168.1.%d", i + 1));
- when(apacheOrg.toString()).thenReturn(String.format("192.168.1.%d", i + 1));
- when(apacheOrg.getHostName()).thenReturn("www.apache.org");
+ final InetAddress apacheOrg = InetAddress.getByName("site" + (i + 1) + ".mock");
resolvedAddresses.add(apacheOrg);
}
@@ -826,10 +838,7 @@ public void testEmptyResolution() throws UnknownHostException {
list.add(InetSocketAddress.createUnresolved("www.google.com", 1234));
final List resolvedAddresses = new ArrayList();
- final InetAddress apacheOrg1 = mock(InetAddress.class);
- when(apacheOrg1.getHostAddress()).thenReturn("192.168.1.1");
- when(apacheOrg1.toString()).thenReturn("www.apache.org");
- when(apacheOrg1.getHostName()).thenReturn("www.apache.org");
+ final InetAddress apacheOrg1 = InetAddress.getByName("site1.mock");
resolvedAddresses.add(apacheOrg1);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java
index 622dadfa77a..3194b6ae94f 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java
@@ -121,7 +121,7 @@ public void testTruncationNullLog() throws Exception {
append(zkdb, i);
}
zkdb.close();
- File[] logs = snaplog.getDataDir().listFiles();
+ File[] logs = snaplog.getDataLogDir().listFiles();
for (int i = 0; i < logs.length; i++) {
LOG.debug("Deleting: {}", logs[i].getName());
assertTrue(logs[i].delete(), "Failed to delete log file: " + logs[i].getName());
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java
index a3d6eef7bdc..4a46a9c2a0f 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java
@@ -21,10 +21,14 @@
import java.io.IOException;
import java.io.PrintWriter;
import java.util.Collections;
+import java.util.List;
import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
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.server.watch.IWatchManager;
import org.apache.zookeeper.server.watch.WatchManagerFactory;
import org.apache.zookeeper.server.watch.WatcherOrBitSet;
@@ -59,12 +63,12 @@ public void removeWatcher(Watcher watcher) {
}
@Override
- public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type) {
+ public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, List acl) {
return new WatcherOrBitSet(Collections.emptySet());
}
@Override
- public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, WatcherOrBitSet suppress) {
+ public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, List acl, WatcherOrBitSet suppress) {
return new WatcherOrBitSet(Collections.emptySet());
}
@@ -120,6 +124,7 @@ public void testBehavior() throws IOException, InterruptedException, KeeperExcep
try (ZooKeeper zk = createClient(hostPort)) {
// the server will generate an exception as our custom watch manager doesn't implement
// the new version of addWatch()
+ zk.create("/foo", null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
zk.addWatch("/foo", event -> {
}, AddWatchMode.PERSISTENT_RECURSIVE);
}