diff --git a/.gitignore b/.gitignore index f8b39ded50d..cf5ff24f045 100644 --- a/.gitignore +++ b/.gitignore @@ -68,6 +68,11 @@ tags .cproject .project obj +zookeeper-server/src/main/resources/lib/ant-eclipse-* +zookeeper-server/src/main/resources/lib/ivy-* +zookeeper-server/src/main/java/org/apache/zookeeper/version/Info.java +zookeeper-server/src/main/java/org/apache/zookeeper/version/VersionInfoMain.java +zookeeper-server/src/test/resources/ zookeeper-client/zookeeper-client-c/Makefile.in zookeeper-client/zookeeper-client-c/aclocal.m4 zookeeper-client/zookeeper-client-c/autom4te.cache/ @@ -80,12 +85,23 @@ zookeeper-client/zookeeper-client-c/depcomp zookeeper-client/zookeeper-client-c/install-sh zookeeper-client/zookeeper-client-c/ltmain.sh zookeeper-client/zookeeper-client-c/missing +zookeeper-client/zookeeper-client-c/.deps/ +zookeeper-client/zookeeper-client-c/.libs/ +zookeeper-client/zookeeper-client-c/Makefile +zookeeper-client/zookeeper-client-c/cli_mt +zookeeper-client/zookeeper-client-c/cli_st +zookeeper-client/zookeeper-client-c/config.h +zookeeper-client/zookeeper-client-c/config.status +zookeeper-client/zookeeper-client-c/libtool +zookeeper-client/zookeeper-client-c/load_gen +zookeeper-client/zookeeper-client-c/stamp-h1 +zookeeper-client/zookeeper-client-c/build +zookeeper-client/zookeeper-client-c/core.* zookeeper-client/zookeeper-client-c/TEST-*.txt +zookeeper-client/zookeeper-client-c/*.la +zookeeper-client/zookeeper-client-c/*.lo +zookeeper-client/zookeeper-client-c/*.o zookeeper-client/zookeeper-client-c/generated/ -zookeeper-server/src/main/resources/lib/ant-eclipse-* -zookeeper-server/src/main/resources/lib/ivy-* -zookeeper-server/src/main/java/org/apache/zookeeper/version/Info.java -zookeeper-server/src/main/java/org/apache/zookeeper/version/VersionInfoMain.java # Python *.py[cod] diff --git a/README_packaging.md b/README_packaging.md index 2ccbba6bdf0..e2a2d47672a 100644 --- a/README_packaging.md +++ b/README_packaging.md @@ -8,14 +8,15 @@ http://bigtop.apache.org/ ## Requirements -- ant (recommended version 1.9.4 or later for concurrent JUnit test execution) or maven to build the java code -- gcc, cppunit and python-setuptools are required to build C and python bindings. +- you need maven to build the java code +- gcc, cppunit, openssl and python-setuptools are required to build C and python bindings. (only needed when using `-Pfull-build`) On RHEL machine: ``` yum install cppunit yum install python-setuptools +yum install openssl openssl-devel ``` On Ubuntu: @@ -23,6 +24,7 @@ On Ubuntu: ``` apt-get install cppunit apt-get install python-setuptools +apt-get install openssl libssl-dev ``` diff --git a/excludeFindBugsFilter.xml b/excludeFindBugsFilter.xml index c836911dbe9..a2965944837 100644 --- a/excludeFindBugsFilter.xml +++ b/excludeFindBugsFilter.xml @@ -7,8 +7,5 @@ - - - diff --git a/pom.xml b/pom.xml index a08bb25e1a3..f492c22fd2e 100755 --- a/pom.xml +++ b/pom.xml @@ -280,7 +280,7 @@ 1.2 4.1.42.Final 9.4.18.v20190429 - 2.9.10 + 2.9.10.1 1.1.1 2.11 1.1.7 @@ -574,7 +574,7 @@ org.owasp dependency-check-maven - 5.2.1 + 5.2.4 org.apache.maven.plugins @@ -808,7 +808,7 @@ src/hashtable/* include/winconfig.h tests/wrappers.opt - tests/quorum.cfg + tests/zoo.cfg tests/wrappers-mt.opt **/c-doc.Doxyfile diff --git a/zookeeper-client/zookeeper-client-c/CMakeLists.txt b/zookeeper-client/zookeeper-client-c/CMakeLists.txt index b010b1685a4..05ae915e46c 100644 --- a/zookeeper-client/zookeeper-client-c/CMakeLists.txt +++ b/zookeeper-client/zookeeper-client-c/CMakeLists.txt @@ -182,6 +182,19 @@ target_link_libraries(zookeeper PUBLIC $<$:rt> # clock_gettime $<$:ws2_32>) # Winsock 2.0 +option(WITH_OPENSSL "openssl directory" OFF) +if(WITH_OPENSSL) + target_compile_definitions(zookeeper PUBLIC HAVE_OPENSSL_H) + include_directories(${WITH_OPENSSL}/include) + link_directories(${WITH_OPENSSL}/lib) + if(WIN32) + target_link_libraries(zookeeper PUBLIC ssleay32 libeay32) + else() + target_link_libraries(zookeeper PUBLIC ssl crypto) + endif() + +endif() + if(WANT_SYNCAPI AND NOT WIN32) find_package(Threads REQUIRED) target_link_libraries(zookeeper PUBLIC Threads::Threads) @@ -224,6 +237,7 @@ if(WANT_SYNCAPI) endif() if(WANT_CPPUNIT) + set (CMAKE_CXX_STANDARD 11) add_executable(zktest ${test_sources}) target_include_directories(zktest PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) diff --git a/zookeeper-client/zookeeper-client-c/Makefile.am b/zookeeper-client/zookeeper-client-c/Makefile.am index a5312a8e9f2..8e9b1ca1ffe 100644 --- a/zookeeper-client/zookeeper-client-c/Makefile.am +++ b/zookeeper-client/zookeeper-client-c/Makefile.am @@ -2,13 +2,21 @@ include $(top_srcdir)/aminclude.am AUTOMAKE_OPTIONS = serial-tests + if SOLARIS SOLARIS_CPPFLAGS = -D_POSIX_PTHREAD_SEMANTICS SOLARIS_LIB_LDFLAGS = -lnsl -lsocket endif -AM_CPPFLAGS = -I${srcdir}/include -I${srcdir}/tests -I${srcdir}/generated $(SOLARIS_CPPFLAGS) + +if WANT_OPENSSL + OPENSSL_CPPFLAGS = -DHAVE_OPENSSL_H -I$(OPENSSL_DIR) + OPENSSL_LIB_LDFLAGS = -lssl -lcrypto +endif + +AM_CPPFLAGS = -I${srcdir}/include -I${srcdir}/tests -I${srcdir}/generated $(SOLARIS_CPPFLAGS) $(OPENSSL_CPPFLAGS) AM_CFLAGS = -Wall -Werror -Wdeclaration-after-statement AM_CXXFLAGS = -Wall $(USEIPV6) +LIB_LDFLAGS = -no-undefined -version-info 2 $(SOLARIS_LIB_LDFLAGS) $(OPENSSL_LIB_LDFLAGS) # Additional flags for coverage testing (if enabled) if ENABLEGCOV @@ -16,8 +24,6 @@ if ENABLEGCOV AM_LDFLAGS = -lgcov endif -LIB_LDFLAGS = -no-undefined -version-info 2 $(SOLARIS_LIB_LDFLAGS) - pkginclude_HEADERS = include/zookeeper.h include/zookeeper_version.h include/zookeeper_log.h include/proto.h include/recordio.h generated/zookeeper.jute.h EXTRA_DIST=LICENSE @@ -107,7 +113,7 @@ TEST_SOURCES = \ tests/ZooKeeperQuorumServer.h \ tests/TestReadOnlyClient.cc \ tests/TestLogClientEnv.cc \ - tests/TestServerRequireClientSASLAuth.cc \ + tests/TestServerRequireClientSASLAuth.cc \ $(NULL) if SOLARIS @@ -121,14 +127,14 @@ check_PROGRAMS = zktest-st TESTS_ENVIRONMENT = ZKROOT=${srcdir}/../.. \ CLASSPATH=$$CLASSPATH:$$CLOVER_HOME/lib/clover*.jar nodist_zktest_st_SOURCES = $(TEST_SOURCES) -zktest_st_LDADD = libzkst.la libhashtable.la $(CPPUNIT_LIBS) -ldl +zktest_st_LDADD = libzkst.la libhashtable.la $(CPPUNIT_LIBS) $(OPENSSL_LIB_LDFLAGS) -ldl zktest_st_CXXFLAGS = -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS) $(USEIPV6) $(SOLARIS_CPPFLAGS) zktest_st_LDFLAGS = -shared $(SYMBOL_WRAPPERS) $(SOLARIS_LIB_LDFLAGS) if WANT_SYNCAPI check_PROGRAMS += zktest-mt nodist_zktest_mt_SOURCES = $(TEST_SOURCES) tests/PthreadMocks.cc - zktest_mt_LDADD = libzkmt.la libhashtable.la -lpthread $(CPPUNIT_LIBS) -ldl + zktest_mt_LDADD = libzkmt.la libhashtable.la -lpthread $(CPPUNIT_LIBS) $(OPENSSL_LIB_LDFLAGS) -ldl zktest_mt_CXXFLAGS = -DUSE_STATIC_LIB -DTHREADED $(CPPUNIT_CFLAGS) $(USEIPV6) if SOLARIS SHELL_SYMBOL_WRAPPERS_MT = cat ${srcdir}/tests/wrappers-mt.opt diff --git a/zookeeper-client/zookeeper-client-c/configure.ac b/zookeeper-client/zookeeper-client-c/configure.ac index 1ecd17dbf8b..f155c3fecb4 100644 --- a/zookeeper-client/zookeeper-client-c/configure.ac +++ b/zookeeper-client/zookeeper-client-c/configure.ac @@ -23,6 +23,7 @@ DX_INIT_DOXYGEN([zookeeper],[c-doc.Doxyfile],[docs]) # initialize automake AM_INIT_AUTOMAKE([-Wall foreign]) + AC_CONFIG_HEADER([config.h]) # Checks for programs. @@ -37,6 +38,26 @@ else CHECK_CPPUNIT(1.10.2) fi +AM_CONDITIONAL([WANT_OPENSSL],[test "x$with_openssl" != x]) + + +AC_ARG_WITH(openssl, + AS_HELP_STRING([--without-openssl], + [Do not use Openssl. Default: auto-detect]), [ +case "$with_openssl" in + yes|no) + : # Nothing special to do here + ;; + *) + if test ! -d "$withval" ; then + AC_MSG_ERROR([--with-openssl path does not point to a directory]) + fi + OPENSSL_DIR="$withval" + AC_SUBST(OPENSSL_DIR) + esac +]) +AH_TEMPLATE(USE_OPENSSL,[Openssl support is available]) + if test "$CALLER" = "ANT" ; then CPPUNIT_CFLAGS="$CPPUNIT_CFLAGS -DZKSERVER_CMD=\"\\\"${base_dir}/zookeeper-client/zookeeper-client-c/tests/zkServer.sh\\\"\"" else @@ -92,6 +113,9 @@ AC_MSG_CHECKING([whether to enable gcov]) AS_IF([test "x${enable_gcov}" = "xyes"],AC_MSG_RESULT([yes]),AC_MSG_RESULT([no])) AM_CONDITIONAL([ENABLEGCOV],[test "x${enable_gcov}" = "xyes"]) + +CXXFLAGS="$CXXFLAGS -std=c++11" + AC_ARG_WITH([syncapi], [AS_HELP_STRING([--with-syncapi],[build with support for SyncAPI [default=yes]])], [],[with_syncapi=yes]) diff --git a/zookeeper-client/zookeeper-client-c/include/win_getopt.h b/zookeeper-client/zookeeper-client-c/include/win_getopt.h new file mode 100644 index 00000000000..c50c7a47140 --- /dev/null +++ b/zookeeper-client/zookeeper-client-c/include/win_getopt.h @@ -0,0 +1,674 @@ +/** + * 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. + */ + +/** + * DISCLAIMER + * This file is part of the mingw-w64 runtime package. + * + * The mingw-w64 runtime package and its code is distributed in the hope that it + * will be useful but WITHOUT ANY WARRANTY. ALL WARRANTIES, EXPRESSED OR + * IMPLIED ARE HEREBY DISCLAIMED. This includes but is not limited to + * warranties of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. + */ + + /* + * Copyright (c) 2002 Todd C. Miller + * + * Permission to use, copy, modify, and distribute this software for any + * purpose with or without fee is hereby granted, provided that the above + * copyright notice and this permission notice appear in all copies. + * + * THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + * WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + * MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + * ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + * WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + * ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + * OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + * + * Sponsored in part by the Defense Advanced Research Projects + * Agency (DARPA) and Air Force Research Laboratory, Air Force + * Materiel Command, USAF, under agreement number F39502-99-1-0512. + */ + +/*- + * Copyright (c) 2000 The NetBSD Foundation, Inc. + * All rights reserved. + * + * This code is derived from software contributed to The NetBSD Foundation + * by Dieter Baron and Thomas Klausner. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE NETBSD FOUNDATION, INC. AND CONTRIBUTORS + * ``AS IS'' AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED + * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR + * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE FOUNDATION OR CONTRIBUTORS + * BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) 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 OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +#ifndef __GETOPT_H__ + +#pragma warning(disable:4996); + +#define __GETOPT_H__ + +/* All the headers include this file. */ +#include +#include +#include +#include +#include +#include +#include + +#ifdef __cplusplus +extern "C" { +#endif + +#define REPLACE_GETOPT /* use this getopt as the system getopt(3) */ + +#ifdef REPLACE_GETOPT +int opterr = 1; /* if error message should be printed */ +int optind = 1; /* index into parent argv vector */ +int optopt = '?'; /* character checked for validity */ +#undef optreset /* see getopt.h */ +#define optreset __mingw_optreset +int optreset; /* reset getopt */ +char *optarg; /* argument associated with option */ +#endif + +//extern int optind; /* index of first non-option in argv */ +//extern int optopt; /* single option character, as parsed */ +//extern int opterr; /* flag to enable built-in diagnostics... */ +// /* (user may set to zero, to suppress) */ +// +//extern char *optarg; /* pointer to argument of current option */ + +#define PRINT_ERROR ((opterr) && (*options != ':')) + +#define FLAG_PERMUTE 0x01 /* permute non-options to the end of argv */ +#define FLAG_ALLARGS 0x02 /* treat non-options as args to option "-1" */ +#define FLAG_LONGONLY 0x04 /* operate as getopt_long_only */ + +/* return values */ +#define BADCH (int)'?' +#define BADARG ((*options == ':') ? (int)':' : (int)'?') +#define INORDER (int)1 + +#ifndef __CYGWIN__ +#define __progname __argv[0] +#else +extern char __declspec(dllimport) *__progname; +#endif + +#ifdef __CYGWIN__ +static char EMSG[] = ""; +#else +#define EMSG "" +#endif + +static int getopt_internal(int, char * const *, const char *, + const struct option *, int *, int); +static int parse_long_options(char * const *, const char *, + const struct option *, int *, int); +static int gcd(int, int); +static void permute_args(int, int, int, char * const *); + +static char *place = EMSG; /* option letter processing */ + +/* XXX: set optreset to 1 rather than these two */ +static int nonopt_start = -1; /* first non option argument (for permute) */ +static int nonopt_end = -1; /* first option after non options (for permute) */ + +/* Error messages */ +static const char recargchar[] = "option requires an argument -- %c"; +static const char recargstring[] = "option requires an argument -- %s"; +static const char ambig[] = "ambiguous option -- %.*s"; +static const char noarg[] = "option doesn't take an argument -- %.*s"; +static const char illoptchar[] = "unknown option -- %c"; +static const char illoptstring[] = "unknown option -- %s"; + +static void +_vwarnx(const char *fmt,va_list ap) +{ + (void)fprintf(stderr,"%s: ",__progname); + if (fmt != NULL) + (void)vfprintf(stderr,fmt,ap); + (void)fprintf(stderr,"\n"); +} + +static void +warnx(const char *fmt,...) +{ + va_list ap; + va_start(ap,fmt); + _vwarnx(fmt,ap); + va_end(ap); +} + +/* + * Compute the greatest common divisor of a and b. + */ +static int +gcd(int a, int b) +{ + int c; + + c = a % b; + while (c != 0) { + a = b; + b = c; + c = a % b; + } + + return (b); +} + +/* + * Exchange the block from nonopt_start to nonopt_end with the block + * from nonopt_end to opt_end (keeping the same order of arguments + * in each block). + */ +static void +permute_args(int panonopt_start, int panonopt_end, int opt_end, + char * const *nargv) +{ + int cstart, cyclelen, i, j, ncycle, nnonopts, nopts, pos; + char *swap; + + /* + * compute lengths of blocks and number and size of cycles + */ + nnonopts = panonopt_end - panonopt_start; + nopts = opt_end - panonopt_end; + ncycle = gcd(nnonopts, nopts); + cyclelen = (opt_end - panonopt_start) / ncycle; + + for (i = 0; i < ncycle; i++) { + cstart = panonopt_end+i; + pos = cstart; + for (j = 0; j < cyclelen; j++) { + if (pos >= panonopt_end) + pos -= nnonopts; + else + pos += nopts; + swap = nargv[pos]; + /* LINTED const cast */ + ((char **) nargv)[pos] = nargv[cstart]; + /* LINTED const cast */ + ((char **)nargv)[cstart] = swap; + } + } +} + +#ifdef REPLACE_GETOPT +/* + * getopt -- + * Parse argc/argv argument vector. + * + * [eventually this will replace the BSD getopt] + */ +int +getopt(int nargc, char * const *nargv, const char *options) +{ + + /* + * We don't pass FLAG_PERMUTE to getopt_internal() since + * the BSD getopt(3) (unlike GNU) has never done this. + * + * Furthermore, since many privileged programs call getopt() + * before dropping privileges it makes sense to keep things + * as simple (and bug-free) as possible. + */ + return (getopt_internal(nargc, nargv, options, NULL, NULL, 0)); +} +#endif /* REPLACE_GETOPT */ + +//extern int getopt(int nargc, char * const *nargv, const char *options); + +#ifdef _BSD_SOURCE +/* + * BSD adds the non-standard `optreset' feature, for reinitialisation + * of `getopt' parsing. We support this feature, for applications which + * proclaim their BSD heritage, before including this header; however, + * to maintain portability, developers are advised to avoid it. + */ +# define optreset __mingw_optreset +extern int optreset; +#endif +#ifdef __cplusplus +} +#endif +/* + * POSIX requires the `getopt' API to be specified in `unistd.h'; + * thus, `unistd.h' includes this header. However, we do not want + * to expose the `getopt_long' or `getopt_long_only' APIs, when + * included in this manner. Thus, close the standard __GETOPT_H__ + * declarations block, and open an additional __GETOPT_LONG_H__ + * specific block, only when *not* __UNISTD_H_SOURCED__, in which + * to declare the extended API. + */ +#endif /* !defined(__GETOPT_H__) */ + +#if !defined(__UNISTD_H_SOURCED__) && !defined(__GETOPT_LONG_H__) +#define __GETOPT_LONG_H__ + +#ifdef __cplusplus +extern "C" { +#endif + +struct option /* specification for a long form option... */ +{ + const char *name; /* option name, without leading hyphens */ + int has_arg; /* does it take an argument? */ + int *flag; /* where to save its status, or NULL */ + int val; /* its associated status value */ +}; + +enum /* permitted values for its `has_arg' field... */ +{ + no_argument = 0, /* option never takes an argument */ + required_argument, /* option always requires an argument */ + optional_argument /* option may take an argument */ +}; + +/* + * parse_long_options -- + * Parse long options in argc/argv argument vector. + * Returns -1 if short_too is set and the option does not match long_options. + */ +static int +parse_long_options(char * const *nargv, const char *options, + const struct option *long_options, int *idx, int short_too) +{ + char *current_argv, *has_equal; + size_t current_argv_len; + int i, ambiguous, match; + +#define IDENTICAL_INTERPRETATION(_x, _y) \ + (long_options[(_x)].has_arg == long_options[(_y)].has_arg && \ + long_options[(_x)].flag == long_options[(_y)].flag && \ + long_options[(_x)].val == long_options[(_y)].val) + + current_argv = place; + match = -1; + ambiguous = 0; + + optind++; + + if ((has_equal = strchr(current_argv, '=')) != NULL) { + /* argument found (--option=arg) */ + current_argv_len = has_equal - current_argv; + has_equal++; + } else + current_argv_len = strlen(current_argv); + + for (i = 0; long_options[i].name; i++) { + /* find matching long option */ + if (strncmp(current_argv, long_options[i].name, + current_argv_len)) + continue; + + if (strlen(long_options[i].name) == current_argv_len) { + /* exact match */ + match = i; + ambiguous = 0; + break; + } + /* + * If this is a known short option, don't allow + * a partial match of a single character. + */ + if (short_too && current_argv_len == 1) + continue; + + if (match == -1) /* partial match */ + match = i; + else if (!IDENTICAL_INTERPRETATION(i, match)) + ambiguous = 1; + } + if (ambiguous) { + /* ambiguous abbreviation */ + if (PRINT_ERROR) + warnx(ambig, (int)current_argv_len, + current_argv); + optopt = 0; + return (BADCH); + } + if (match != -1) { /* option found */ + if (long_options[match].has_arg == no_argument + && has_equal) { + if (PRINT_ERROR) + warnx(noarg, (int)current_argv_len, + current_argv); + /* + * XXX: GNU sets optopt to val regardless of flag + */ + if (long_options[match].flag == NULL) + optopt = long_options[match].val; + else + optopt = 0; + return (BADARG); + } + if (long_options[match].has_arg == required_argument || + long_options[match].has_arg == optional_argument) { + if (has_equal) + optarg = has_equal; + else if (long_options[match].has_arg == + required_argument) { + /* + * optional argument doesn't use next nargv + */ + optarg = nargv[optind++]; + } + } + if ((long_options[match].has_arg == required_argument) + && (optarg == NULL)) { + /* + * Missing argument; leading ':' indicates no error + * should be generated. + */ + if (PRINT_ERROR) + warnx(recargstring, + current_argv); + /* + * XXX: GNU sets optopt to val regardless of flag + */ + if (long_options[match].flag == NULL) + optopt = long_options[match].val; + else + optopt = 0; + --optind; + return (BADARG); + } + } else { /* unknown option */ + if (short_too) { + --optind; + return (-1); + } + if (PRINT_ERROR) + warnx(illoptstring, current_argv); + optopt = 0; + return (BADCH); + } + if (idx) + *idx = match; + if (long_options[match].flag) { + *long_options[match].flag = long_options[match].val; + return (0); + } else + return (long_options[match].val); +#undef IDENTICAL_INTERPRETATION +} + +/* + * getopt_internal -- + * Parse argc/argv argument vector. Called by user level routines. + */ +static int +getopt_internal(int nargc, char * const *nargv, const char *options, + const struct option *long_options, int *idx, int flags) +{ + char *oli; /* option letter list index */ + int optchar, short_too; + static int posixly_correct = -1; + + if (options == NULL) + return (-1); + + /* + * XXX Some GNU programs (like cvs) set optind to 0 instead of + * XXX using optreset. Work around this braindamage. + */ + if (optind == 0) + optind = optreset = 1; + + /* + * Disable GNU extensions if POSIXLY_CORRECT is set or options + * string begins with a '+'. + * + * CV, 2009-12-14: Check POSIXLY_CORRECT anew if optind == 0 or + * optreset != 0 for GNU compatibility. + */ + if (posixly_correct == -1 || optreset != 0) + posixly_correct = (getenv("POSIXLY_CORRECT") != NULL); + if (*options == '-') + flags |= FLAG_ALLARGS; + else if (posixly_correct || *options == '+') + flags &= ~FLAG_PERMUTE; + if (*options == '+' || *options == '-') + options++; + + optarg = NULL; + if (optreset) + nonopt_start = nonopt_end = -1; +start: + if (optreset || !*place) { /* update scanning pointer */ + optreset = 0; + if (optind >= nargc) { /* end of argument vector */ + place = EMSG; + if (nonopt_end != -1) { + /* do permutation, if we have to */ + permute_args(nonopt_start, nonopt_end, + optind, nargv); + optind -= nonopt_end - nonopt_start; + } + else if (nonopt_start != -1) { + /* + * If we skipped non-options, set optind + * to the first of them. + */ + optind = nonopt_start; + } + nonopt_start = nonopt_end = -1; + return (-1); + } + if (*(place = nargv[optind]) != '-' || + (place[1] == '\0' && strchr(options, '-') == NULL)) { + place = EMSG; /* found non-option */ + if (flags & FLAG_ALLARGS) { + /* + * GNU extension: + * return non-option as argument to option 1 + */ + optarg = nargv[optind++]; + return (INORDER); + } + if (!(flags & FLAG_PERMUTE)) { + /* + * If no permutation wanted, stop parsing + * at first non-option. + */ + return (-1); + } + /* do permutation */ + if (nonopt_start == -1) + nonopt_start = optind; + else if (nonopt_end != -1) { + permute_args(nonopt_start, nonopt_end, + optind, nargv); + nonopt_start = optind - + (nonopt_end - nonopt_start); + nonopt_end = -1; + } + optind++; + /* process next argument */ + goto start; + } + if (nonopt_start != -1 && nonopt_end == -1) + nonopt_end = optind; + + /* + * If we have "-" do nothing, if "--" we are done. + */ + if (place[1] != '\0' && *++place == '-' && place[1] == '\0') { + optind++; + place = EMSG; + /* + * We found an option (--), so if we skipped + * non-options, we have to permute. + */ + if (nonopt_end != -1) { + permute_args(nonopt_start, nonopt_end, + optind, nargv); + optind -= nonopt_end - nonopt_start; + } + nonopt_start = nonopt_end = -1; + return (-1); + } + } + + /* + * Check long options if: + * 1) we were passed some + * 2) the arg is not just "-" + * 3) either the arg starts with -- we are getopt_long_only() + */ + if (long_options != NULL && place != nargv[optind] && + (*place == '-' || (flags & FLAG_LONGONLY))) { + short_too = 0; + if (*place == '-') + place++; /* --foo long option */ + else if (*place != ':' && strchr(options, *place) != NULL) + short_too = 1; /* could be short option too */ + + optchar = parse_long_options(nargv, options, long_options, + idx, short_too); + if (optchar != -1) { + place = EMSG; + return (optchar); + } + } + + if ((optchar = (int)*place++) == (int)':' || + (optchar == (int)'-' && *place != '\0') || + (oli = (char*)strchr(options, optchar)) == NULL) { + /* + * If the user specified "-" and '-' isn't listed in + * options, return -1 (non-option) as per POSIX. + * Otherwise, it is an unknown option character (or ':'). + */ + if (optchar == (int)'-' && *place == '\0') + return (-1); + if (!*place) + ++optind; + if (PRINT_ERROR) + warnx(illoptchar, optchar); + optopt = optchar; + return (BADCH); + } + if (long_options != NULL && optchar == 'W' && oli[1] == ';') { + /* -W long-option */ + if (*place) /* no space */ + /* NOTHING */; + else if (++optind >= nargc) { /* no arg */ + place = EMSG; + if (PRINT_ERROR) + warnx(recargchar, optchar); + optopt = optchar; + return (BADARG); + } else /* white space */ + place = nargv[optind]; + optchar = parse_long_options(nargv, options, long_options, + idx, 0); + place = EMSG; + return (optchar); + } + if (*++oli != ':') { /* doesn't take argument */ + if (!*place) + ++optind; + } else { /* takes (optional) argument */ + optarg = NULL; + if (*place) /* no white space */ + optarg = place; + else if (oli[1] != ':') { /* arg not optional */ + if (++optind >= nargc) { /* no arg */ + place = EMSG; + if (PRINT_ERROR) + warnx(recargchar, optchar); + optopt = optchar; + return (BADARG); + } else + optarg = nargv[optind]; + } + place = EMSG; + ++optind; + } + /* dump back option letter */ + return (optchar); +} + +/* + * getopt_long -- + * Parse argc/argv argument vector. + */ +int +getopt_long(int nargc, char * const *nargv, const char *options, + const struct option *long_options, int *idx) +{ + + return (getopt_internal(nargc, nargv, options, long_options, idx, + FLAG_PERMUTE)); +} + +/* + * getopt_long_only -- + * Parse argc/argv argument vector. + */ +int +getopt_long_only(int nargc, char * const *nargv, const char *options, + const struct option *long_options, int *idx) +{ + + return (getopt_internal(nargc, nargv, options, long_options, idx, + FLAG_PERMUTE|FLAG_LONGONLY)); +} + +//extern int getopt_long(int nargc, char * const *nargv, const char *options, +// const struct option *long_options, int *idx); +//extern int getopt_long_only(int nargc, char * const *nargv, const char *options, +// const struct option *long_options, int *idx); +/* + * Previous MinGW implementation had... + */ +#ifndef HAVE_DECL_GETOPT +/* + * ...for the long form API only; keep this for compatibility. + */ +# define HAVE_DECL_GETOPT 1 +#endif + +#ifdef __cplusplus +} +#endif + +#endif /* !defined(__UNISTD_H_SOURCED__) && !defined(__GETOPT_LONG_H__) */ diff --git a/zookeeper-client/zookeeper-client-c/include/zookeeper.h b/zookeeper-client/zookeeper-client-c/include/zookeeper.h index 304dafe2f69..3a6a689a009 100644 --- a/zookeeper-client/zookeeper-client-c/include/zookeeper.h +++ b/zookeeper-client/zookeeper-client-c/include/zookeeper.h @@ -32,6 +32,10 @@ #include /* for struct sock_addr and socklen_t */ #endif +#ifdef HAVE_OPENSSL_H +#include +#endif + #include #include @@ -106,6 +110,7 @@ enum ZOO_ERRORS { ZRECONFIGINPROGRESS = -14, /*!< Reconfiguration requested while another reconfiguration is currently in progress. This is currently not supported. Please retry. */ + ZSSLCONNECTIONERROR = -15, /*!< The SSL connection Error */ /** API errors. * This is never thrown by the server, it shouldn't be used other than @@ -284,6 +289,34 @@ extern ZOOAPI const int ZOO_NOTWATCHING_EVENT; */ typedef struct _zhandle zhandle_t; +/** + * This structure represents the certificates to zookeeper. + */ +typedef struct _zcert { + char *certstr; + char *ca; + char *cert; + char *key; + char *passwd; +} zcert_t; + +/** + * This structure represents the socket to zookeeper. + */ +typedef struct _zsock { +#ifdef WIN32 + SOCKET sock; +#else + int sock; +#endif + zcert_t *cert; +#ifdef HAVE_OPENSSL_H + SSL *ssl_sock; + SSL_CTX *ssl_ctx; +#endif +} zsock_t; + + /** * \brief client id structure. * @@ -496,6 +529,13 @@ typedef void (*log_callback_fn)(const char *message); ZOOAPI zhandle_t *zookeeper_init(const char *host, watcher_fn fn, int recv_timeout, const clientid_t *clientid, void *context, int flags); +#ifdef HAVE_OPENSSL_H +ZOOAPI zhandle_t *zookeeper_init_ssl(const char *host, const char *cert, watcher_fn fn, + int recv_timeout, const clientid_t *clientid, void *context, int flags); +#endif + +ZOOAPI void close_zsock(zsock_t *zsock); + /** * \brief create a handle to communicate with zookeeper. * diff --git a/zookeeper-client/zookeeper-client-c/pom.xml b/zookeeper-client/zookeeper-client-c/pom.xml index e3703f1f85d..44202296d92 100755 --- a/zookeeper-client/zookeeper-client-c/pom.xml +++ b/zookeeper-client/zookeeper-client-c/pom.xml @@ -88,6 +88,7 @@ ANT + --with-openssl=/usr/include/openssl/ --prefix=${project.build.directory}/c ${c-test-coverage-arg} diff --git a/zookeeper-client/zookeeper-client-c/src/cli.c b/zookeeper-client/zookeeper-client-c/src/cli.c index 181cadf0317..e8151ec8c66 100644 --- a/zookeeper-client/zookeeper-client-c/src/cli.c +++ b/zookeeper-client/zookeeper-client-c/src/cli.c @@ -34,12 +34,14 @@ #include #include #include +#include #else #include "winport.h" //#include <-- can't include, conflicting definitions of close() int read(int _FileHandle, void * _DstBuf, unsigned int _MaxCharCount); int write(int _Filehandle, const void * _Buf, unsigned int _MaxCharCount); #define ctime_r(tctime, buffer) ctime_s (buffer, 40, tctime) +#include "win_getopt.h" // VisualStudio doesn't contain 'getopt' #endif #include @@ -56,7 +58,8 @@ static zhandle_t *zh; static clientid_t myid; static const char *clientIdFile = 0; struct timeval startTime; -static char cmd[1024]; +static char *cmd; +static char *cert; static int batchMode=0; static int to_send=0; @@ -741,6 +744,15 @@ int handleBatchMode(char* arg, char* buf, size_t maxlen) { } int main(int argc, char **argv) { + static struct option long_options[] = { + {"host", required_argument, NULL, 'h'}, //hostPort + {"ssl", required_argument, NULL, 's'}, //certificate files + {"myid", required_argument, NULL, 'm'}, //myId file + {"cmd", required_argument, NULL, 'c'}, //cmd + {"readonly", no_argument, NULL, 'r'}, //read-only + {"debug", no_argument, NULL, 'd'}, //set log level to DEBUG from the beginning + {NULL, 0, NULL, 0}, + }; #ifndef THREADED fd_set rfds, wfds, efds; int processed=0; @@ -752,46 +764,82 @@ int main(int argc, char **argv) { char appId[64]; #endif int bufoff = 0; - int flags, i; + int flags; FILE *fh; - if (argc < 2) { + int opt; + int option_index = 0; + + verbose = 0; + zoo_set_debug_level(ZOO_LOG_LEVEL_WARN); + + flags = 0; + while ((opt = getopt_long(argc, argv, "h:s:m:c:rd", long_options, &option_index)) != -1) { + switch (opt) { + case 'h': + hostPort = strdup(optarg); + break; + case 'm': + clientIdFile = strdup(optarg); + fh = fopen(clientIdFile, "r"); + if (fh) { + if (fread(&myid, sizeof(myid), 1, fh) != sizeof(myid)) { + memset(&myid, 0, sizeof(myid)); + } + fclose(fh); + } + break; + case 'r': + flags = ZOO_READONLY; + break; + case 'c': + cmd = strdup(optarg); + batchMode = 1; + fprintf(stderr,"Batch mode: %s\n",cmd); + break; + case 's': + cert = strdup(optarg); + break; + case 'd': + verbose = 1; + zoo_set_debug_level(ZOO_LOG_LEVEL_DEBUG); + fprintf(stderr, "logging level set to DEBUG\n"); + break; + case '?': + if (optopt == 'h') { + fprintf (stderr, "Option -%c requires host list.\n", optopt); + } else if (isprint (optopt)) { + fprintf (stderr, "Unknown option `-%c'.\n", optopt); + } else { + fprintf (stderr, + "Unknown option character `\\x%x'.\n", + optopt); + return 1; + } + } + } + + if (!hostPort) { fprintf(stderr, - "USAGE %s zookeeper_host_list [clientid_file|cmd:(ls|ls2|create|create2|od|...)]\n", + "\nUSAGE: %s -h zk_host_1:port_1,zk_host_2:port_2,... [OPTIONAL ARGS]\n\n" + "MANDATORY ARGS:\n" + "-h, --host Comma separated list of ZooKeeper host:port pairs\n\n" + "OPTIONAL ARGS:\n" + "-m, --myid Path to the file contains the client ID\n" + "-c, --cmd Command to execute, e.g. ls|ls2|create|create2|od|...\n" + "-s, --ssl Comma separated parameters to initiate SSL connection\n" + " e.g.: server_cert.crt,client_cert.crt,client_priv_key.pem,passwd\n" + "-r, --readonly Connect in read-only mode\n" + "-d, --debug Activate debug logs right from the beginning (you can also use the \n" + " command 'verbose' later to activate debug logs in the cli shell)\n\n", argv[0]); fprintf(stderr, - "Version: ZooKeeper cli (c client) version %d.%d.%d\n", + "Version: ZooKeeper cli (c client) version %d.%d.%d\n", ZOO_MAJOR_VERSION, ZOO_MINOR_VERSION, ZOO_PATCH_VERSION); return 2; } - if (argc > 2) { - int batchModeRes = handleBatchMode(argv[2], cmd, sizeof(cmd)); - if (batchModeRes == -1) { - return 2; - } else if(batchModeRes == 1){ - batchMode=1; - fprintf(stderr,"Batch mode: '%s'\n",cmd); - }else{ - clientIdFile = argv[2]; - fh = fopen(clientIdFile, "r"); - if (fh) { - if (fread(&myid, sizeof(myid), 1, fh) != sizeof(myid)) { - memset(&myid, 0, sizeof(myid)); - } - fclose(fh); - } - } - } - - flags = 0; - for (i = 1; i < argc; ++i) { - if (strcmp("-r", argv[i]) == 0) { - flags = ZOO_READONLY; - break; - } - } #ifdef YCA strcpy(appId,"yahoo.example.yca_test"); @@ -807,11 +855,18 @@ int main(int argc, char **argv) { #else strcpy(p, "dummy"); #endif - verbose = 0; - zoo_set_debug_level(ZOO_LOG_LEVEL_WARN); zoo_deterministic_conn_order(1); // enable deterministic order - hostPort = argv[1]; + +#ifdef HAVE_OPENSSL_H + if (!cert) { + zh = zookeeper_init(hostPort, watcher, 30000, &myid, NULL, flags); + } else { + zh = zookeeper_init_ssl(hostPort, cert, watcher, 30000, &myid, NULL, flags); + } +#else zh = zookeeper_init(hostPort, watcher, 30000, &myid, NULL, flags); +#endif + if (!zh) { return errno; } diff --git a/zookeeper-client/zookeeper-client-c/src/st_adaptor.c b/zookeeper-client/zookeeper-client-c/src/st_adaptor.c index 5e9a4ff9965..ddc2582db34 100644 --- a/zookeeper-client/zookeeper-client-c/src/st_adaptor.c +++ b/zookeeper-client/zookeeper-client-c/src/st_adaptor.c @@ -16,7 +16,7 @@ * limitations under the License. */ -#ifndef DLL_EXPORT +#if !defined(DLL_EXPORT) && !defined(USE_STATIC_LIB) # define USE_STATIC_LIB #endif diff --git a/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h b/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h index 97995e36ace..8157472dd4b 100644 --- a/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h +++ b/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h @@ -43,6 +43,7 @@ #define ASSOCIATING_STATE_DEF 2 #define CONNECTED_STATE_DEF 3 #define READONLY_STATE_DEF 5 +#define SSL_CONNECTING_STATE_DEF 7 #define NOTCONNECTED_STATE_DEF 999 /* zookeeper event type constants */ @@ -185,11 +186,7 @@ typedef struct _auth_list_head { * This structure represents the connection to zookeeper. */ struct _zhandle { -#ifdef WIN32 - SOCKET fd; // the descriptor used to talk to zookeeper -#else - int fd; // the descriptor used to talk to zookeeper -#endif + zsock_t *fd; // Hostlist and list of addresses char *hostname; // hostname contains list of zookeeper servers to connect to diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index 96d661683ed..7ab5eed33cb 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -79,9 +79,15 @@ #include #endif +#ifdef HAVE_OPENSSL_H +#include +#include +#endif + #ifdef __MACH__ // OS X #include #include +#include #endif #ifdef WIN32 @@ -124,6 +130,7 @@ const int ZOO_CONNECTING_STATE = CONNECTING_STATE_DEF; const int ZOO_ASSOCIATING_STATE = ASSOCIATING_STATE_DEF; const int ZOO_CONNECTED_STATE = CONNECTED_STATE_DEF; const int ZOO_READONLY_STATE = READONLY_STATE_DEF; +const int ZOO_SSL_CONNECTING_STATE = SSL_CONNECTING_STATE_DEF; const int ZOO_NOTCONNECTED_STATE = NOTCONNECTED_STATE_DEF; static __attribute__ ((unused)) const char* state2String(int state){ @@ -132,6 +139,8 @@ static __attribute__ ((unused)) const char* state2String(int state){ return "ZOO_CLOSED_STATE"; case CONNECTING_STATE_DEF: return "ZOO_CONNECTING_STATE"; + case SSL_CONNECTING_STATE_DEF: + return "ZOO_SSL_CONNECTING_STATE"; case ASSOCIATING_STATE_DEF: return "ZOO_ASSOCIATING_STATE"; case CONNECTED_STATE_DEF: @@ -273,6 +282,10 @@ static struct sockaddr_storage *addr_rw_server = 0; static void *SYNCHRONOUS_MARKER = (void*)&SYNCHRONOUS_MARKER; static int isValidPath(const char* path, const int mode); +#ifdef HAVE_OPENSSL_H +static int init_ssl_for_handler(zhandle_t *zh); +static int init_ssl_for_socket(zsock_t *fd, zhandle_t *zh, int fail_on_error); +#endif static int aremove_watches( zhandle_t *zh, const char *path, ZooWatcherType wtype, @@ -323,9 +336,22 @@ static void abort_singlethreaded(zhandle_t *zh) abort(); } -static sendsize_t zookeeper_send(socket_t s, const void* buf, size_t len) +static ssize_t zookeeper_send(zsock_t *fd, const void* buf, size_t len) +{ +#ifdef HAVE_OPENSSL_H + if (fd->ssl_sock) + return (ssize_t)SSL_write(fd->ssl_sock, buf, (int)len); +#endif + return send(fd->sock, buf, len, SEND_FLAGS); +} + +static ssize_t zookeeper_recv(zsock_t *fd, void *buf, size_t len, int flags) { - return send(s, buf, len, SEND_FLAGS); +#ifdef HAVE_OPENSSL_H + if (fd->ssl_sock) + return (ssize_t)SSL_read(fd->ssl_sock, buf, (int)len); +#endif + return recv(fd->sock, buf, len, flags); } /** @@ -360,7 +386,7 @@ void get_system_time(struct timeval *tv) // Default to gettimeofday in case of failure. ret = gettimeofday(tv, NULL); } -#elif CLOCK_MONOTONIC_RAW +#elif defined CLOCK_MONOTONIC_RAW // On Linux, CLOCK_MONOTONIC is affected by ntp slew but CLOCK_MONOTONIC_RAW // is not. We want the non-slewed (constant rate) CLOCK_MONOTONIC_RAW if it // is available. @@ -555,6 +581,22 @@ zk_hashtable *child_result_checker(zhandle_t *zh, int rc) return rc==ZOK ? zh->active_child_watchers : 0; } +void close_zsock(zsock_t *fd) +{ + if (fd->sock != -1) { +#ifdef HAVE_OPENSSL_H + if (fd->ssl_sock) { + SSL_free(fd->ssl_sock); + fd->ssl_sock = NULL; + SSL_CTX_free(fd->ssl_ctx); + fd->ssl_ctx = NULL; + } +#endif + close(fd->sock); + fd->sock = -1; + } +} + /** * Frees and closes everything associated with a handle, * including the handle itself. @@ -573,9 +615,8 @@ static void destroy(zhandle_t *zh) free(zh->hostname); zh->hostname = NULL; } - if (zh->fd != -1) { - close(zh->fd); - zh->fd = -1; + if (zh->fd->sock != -1) { + close_zsock(zh->fd); memset(&zh->addr_cur, 0, sizeof(zh->addr_cur)); zh->state = 0; } @@ -585,7 +626,13 @@ static void destroy(zhandle_t *zh) free(zh->chroot); zh->chroot = NULL; } - +#ifdef HAVE_OPENSSL_H + if (zh->fd->cert) { + free(zh->fd->cert->certstr); + free(zh->fd->cert); + zh->fd->cert = NULL; + } +#endif free_auth_info(&zh->auth_h); destroy_zk_hashtable(zh->active_node_watchers); destroy_zk_hashtable(zh->active_exist_watchers); @@ -1030,10 +1077,9 @@ int update_addrs(zhandle_t *zh) // If we need to do a reconfig and we're currently connected to a server, // then force close that connection so on next interest() call we'll make a // new connection - if (zh->reconfig == 1 && zh->fd != -1) + if (zh->reconfig == 1 && zh->fd->sock != -1) { - close(zh->fd); - zh->fd = -1; + close_zsock(zh->fd); zh->state = ZOO_NOTCONNECTED_STATE; } @@ -1080,7 +1126,7 @@ struct sockaddr* zookeeper_get_connected_host(zhandle_t *zh, if (zh->state!=ZOO_CONNECTED_STATE) { return NULL; } - if (getpeername(zh->fd, addr, addr_len)==-1) { + if (getpeername(zh->fd->sock, addr, addr_len)==-1) { return NULL; } return addr; @@ -1151,7 +1197,7 @@ static void log_env(zhandle_t *zh) { */ static zhandle_t *zookeeper_init_internal(const char *host, watcher_fn watcher, int recv_timeout, const clientid_t *clientid, void *context, int flags, - log_callback_fn log_callback) + log_callback_fn log_callback, zcert_t *cert) { int errnosave = 0; zhandle_t *zh = NULL; @@ -1170,6 +1216,13 @@ static zhandle_t *zookeeper_init_internal(const char *host, watcher_fn watcher, log_env(zh); } + zh->fd = calloc(1, sizeof(zsock_t)); + zh->fd->sock = -1; + if (cert) { + zh->fd->cert = calloc(1, sizeof(zcert_t)); + memcpy(zh->fd->cert, cert, sizeof(zcert_t)); + } + #ifdef _WIN32 if (Win32WSAStartup()){ LOG_ERROR(LOGCALLBACK(zh), "Error initializing ws2_32.dll"); @@ -1188,7 +1241,6 @@ static zhandle_t *zookeeper_init_internal(const char *host, watcher_fn watcher, flags); zh->hostname = NULL; - zh->fd = -1; zh->state = ZOO_NOTCONNECTED_STATE; zh->context = context; zh->recv_timeout = recv_timeout; @@ -1264,6 +1316,7 @@ static zhandle_t *zookeeper_init_internal(const char *host, watcher_fn watcher, abort: errnosave=errno; destroy(zh); + free(zh->fd); free(zh); errno=errnosave; return 0; @@ -1272,16 +1325,30 @@ static zhandle_t *zookeeper_init_internal(const char *host, watcher_fn watcher, zhandle_t *zookeeper_init(const char *host, watcher_fn watcher, int recv_timeout, const clientid_t *clientid, void *context, int flags) { - return zookeeper_init_internal(host, watcher, recv_timeout, clientid, context, flags, NULL); + return zookeeper_init_internal(host, watcher, recv_timeout, clientid, context, flags, NULL, NULL); } zhandle_t *zookeeper_init2(const char *host, watcher_fn watcher, int recv_timeout, const clientid_t *clientid, void *context, int flags, log_callback_fn log_callback) { - return zookeeper_init_internal(host, watcher, recv_timeout, clientid, context, flags, log_callback); + return zookeeper_init_internal(host, watcher, recv_timeout, clientid, context, flags, log_callback, NULL); } +#ifdef HAVE_OPENSSL_H +zhandle_t *zookeeper_init_ssl(const char *host, const char *cert, watcher_fn watcher, + int recv_timeout, const clientid_t *clientid, void *context, int flags) +{ + zcert_t zcert; + zcert.certstr = strdup(cert); + zcert.ca = strtok(strdup(cert), ","); + zcert.cert = strtok(NULL, ","); + zcert.key = strtok(NULL, ","); + zcert.passwd = strtok(NULL, ","); + return zookeeper_init_internal(host, watcher, recv_timeout, clientid, context, flags, NULL, &zcert); +} +#endif + /** * Set a new list of zk servers to connect to. Disconnect will occur if * current connection endpoint is not in the list. @@ -1568,7 +1635,7 @@ static __attribute__ ((unused)) int get_queue_len(buffer_head_t *list) * 0 if send would block while sending the buffer (or a send was incomplete), * 1 if success */ -static int send_buffer(socket_t fd, buffer_list_t *buff) +static int send_buffer(zhandle_t *zh, buffer_list_t *buff) { int len = buff->len; int off = buff->curr_offset; @@ -1578,7 +1645,7 @@ static int send_buffer(socket_t fd, buffer_list_t *buff) /* we need to send the length at the beginning */ int nlen = htonl(len); char *b = (char*)&nlen; - rc = zookeeper_send(fd, b + off, sizeof(nlen) - off); + rc = zookeeper_send(zh->fd, b + off, sizeof(nlen) - off); if (rc == -1) { #ifdef _WIN32 if (WSAGetLastError() != WSAEWOULDBLOCK) { @@ -1597,7 +1664,7 @@ static int send_buffer(socket_t fd, buffer_list_t *buff) if (off >= 4) { /* want off to now represent the offset into the buffer */ off -= sizeof(buff->len); - rc = zookeeper_send(fd, buff->buffer + off, len - off); + rc = zookeeper_send(zh->fd, buff->buffer + off, len - off); if (rc == -1) { #ifdef _WIN32 if (WSAGetLastError() != WSAEWOULDBLOCK) { @@ -1626,7 +1693,7 @@ static int recv_buffer(zhandle_t *zh, buffer_list_t *buff) /* if buffer is less than 4, we are reading in the length */ if (off < 4) { char *buffer = (char*)&(buff->len); - rc = recv(zh->fd, buffer+off, sizeof(int)-off, 0); + rc = zookeeper_recv(zh->fd, buffer+off, sizeof(int)-off, 0); switch (rc) { case 0: errno = EHOSTDOWN; @@ -1652,7 +1719,7 @@ static int recv_buffer(zhandle_t *zh, buffer_list_t *buff) /* want off to now represent the offset into the buffer */ off -= sizeof(buff->len); - rc = recv(zh->fd, buff->buffer+off, buff->len-off, 0); + rc = zookeeper_recv(zh->fd, buff->buffer+off, buff->len-off, 0); /* dirty hack to make new client work against old server * old server sends 40 bytes to finish connection handshake, @@ -1772,7 +1839,7 @@ static int is_connected(zhandle_t* zh) static void cleanup(zhandle_t *zh,int rc) { - close(zh->fd); + close_zsock(zh->fd); if (is_unrecoverable(zh)) { LOG_DEBUG(LOGCALLBACK(zh), "Calling a watcher for a ZOO_SESSION_EVENT and the state=%s", state2String(zh->state)); @@ -1782,7 +1849,6 @@ static void cleanup(zhandle_t *zh,int rc) PROCESS_SESSION_EVENT(zh, ZOO_CONNECTING_STATE); } cleanup_bufs(zh,1,rc); - zh->fd = -1; LOG_DEBUG(LOGCALLBACK(zh), "Previous connection=%s delay=%d", zoo_get_current_server(zh), zh->delay); @@ -2038,6 +2104,11 @@ static int prime_connection(zhandle_t *zh) int len = sizeof(buffer_req); int hlen = 0; struct connect_req req; + + if (zh->state == ZOO_SSL_CONNECTING_STATE) { + // The SSL connection is yet to happen. + return ZOK; + } req.protocolVersion = 0; req.sessionId = zh->seen_rw_server_before ? zh->client_id.client_id : 0; req.passwd_len = sizeof(req.passwd); @@ -2120,7 +2191,7 @@ const int MIN_RW_TIMEOUT = 200; static int ping_rw_server(zhandle_t* zh) { char buf[10]; - socket_t sock; + zsock_t fd; int rc; sendsize_t ssize; int sock_flags; @@ -2132,27 +2203,41 @@ static int ping_rw_server(zhandle_t* zh) #else sock_flags = SOCK_STREAM; #endif - sock = socket(zh->addr_rw_server.ss_family, sock_flags, 0); - if (sock < 0) { + fd.sock = socket(zh->addr_rw_server.ss_family, sock_flags, 0); + if (fd.sock < 0) { return 0; } - zookeeper_set_sock_nodelay(zh, sock); - zookeeper_set_sock_timeout(zh, sock, 1); + zookeeper_set_sock_nodelay(zh, fd.sock); + zookeeper_set_sock_timeout(zh, fd.sock, 1); - rc = zookeeper_connect(zh, &zh->addr_rw_server, sock); + rc = zookeeper_connect(zh, &zh->addr_rw_server, fd.sock); if (rc < 0) { return 0; } - ssize = zookeeper_send(sock, "isro", 4); +#ifdef HAVE_OPENSSL_H + fd.ssl_sock = NULL; + fd.ssl_ctx = NULL; + + if (zh->fd->cert != NULL) { + fd.cert = zh->fd->cert; + rc = init_ssl_for_socket(&fd, zh, 0); + if (rc != ZOK) { + rc = 0; + goto out; + } + } +#endif + + ssize = zookeeper_send(&fd, "isro", 4); if (ssize < 0) { rc = 0; goto out; } memset(buf, 0, sizeof(buf)); - rc = recv(sock, buf, sizeof(buf), 0); + rc = zookeeper_recv(&fd, buf, sizeof(buf), 0); if (rc < 0) { rc = 0; goto out; @@ -2161,7 +2246,7 @@ static int ping_rw_server(zhandle_t* zh) rc = strcmp("rw", buf) == 0; out: - close(sock); + close_zsock(&fd); addr_rw_server = rc ? &zh->addr_rw_server : 0; return rc; } @@ -2292,7 +2377,7 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, return api_epilog(zh, rc); } - *fd = zh->fd; + *fd = zh->fd->sock; *interest = 0; tv->tv_sec = 0; tv->tv_usec = 0; @@ -2322,8 +2407,8 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, // No need to delay -- grab the next server and attempt connection zoo_cycle_next_server(zh); } - zh->fd = socket(zh->addr_cur.ss_family, sock_flags, 0); - if (zh->fd < 0) { + zh->fd->sock = socket(zh->addr_cur.ss_family, sock_flags, 0); + if (zh->fd->sock < 0) { rc = handle_socket_error_msg(zh, __LINE__, ZSYSTEMERROR, @@ -2331,17 +2416,21 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, return api_epilog(zh, rc); } - zookeeper_set_sock_nodelay(zh, zh->fd); - zookeeper_set_sock_noblock(zh, zh->fd); + zookeeper_set_sock_nodelay(zh, zh->fd->sock); + zookeeper_set_sock_noblock(zh, zh->fd->sock); - rc = zookeeper_connect(zh, &zh->addr_cur, zh->fd); + rc = zookeeper_connect(zh, &zh->addr_cur, zh->fd->sock); if (rc == -1) { /* we are handling the non-blocking connect according to * the description in section 16.3 "Non-blocking connect" * in UNIX Network Programming vol 1, 3rd edition */ if (errno == EWOULDBLOCK || errno == EINPROGRESS) { - zh->state = ZOO_CONNECTING_STATE; + // For SSL, we first go to ZOO_SSL_CONNECTING_STATE + if (zh->fd->cert != NULL) + zh->state = ZOO_SSL_CONNECTING_STATE; + else + zh->state = ZOO_CONNECTING_STATE; } else { rc = handle_socket_error_msg(zh, __LINE__, @@ -2350,6 +2439,14 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, return api_epilog(zh, rc); } } else { +#ifdef HAVE_OPENSSL_H + if (zh->fd->cert != NULL) { + // We do SSL_connect() here + if (init_ssl_for_handler(zh) != ZOK) { + return ZSSLCONNECTIONERROR; + } + } +#endif rc = prime_connection(zh); if (rc != 0) { return api_epilog(zh,rc); @@ -2361,7 +2458,7 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, } *tv = get_timeval(zh->recv_timeout/3); } - *fd = zh->fd; + *fd = zh->fd->sock; zh->last_recv = now; zh->last_send = now; zh->last_ping = now; @@ -2369,13 +2466,13 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, zh->ping_rw_timeout = MIN_RW_TIMEOUT; } - if (zh->fd != -1) { + if (zh->fd->sock != -1) { int idle_recv = calculate_interval(&zh->last_recv, &now); int idle_send = calculate_interval(&zh->last_send, &now); int recv_to = zh->recv_timeout*2/3 - idle_recv; int send_to = zh->recv_timeout/3; // have we exceeded the receive timeout threshold? - if (recv_to <= 0) { + if (recv_to <= 0 && zh->state != ZOO_SSL_CONNECTING_STATE) { // We gotta cut our losses and connect to someone else #ifdef _WIN32 errno = WSAETIMEDOUT; @@ -2444,21 +2541,186 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, /* we are interested in a write if we are connected and have something * to send, or we are waiting for a connect to finish. */ if ((zh->to_send.head && is_connected(zh)) - || zh->state == ZOO_CONNECTING_STATE) { + || zh->state == ZOO_CONNECTING_STATE + || zh->state == ZOO_SSL_CONNECTING_STATE) { *interest |= ZOOKEEPER_WRITE; } } return api_epilog(zh,ZOK); } +#ifdef HAVE_OPENSSL_H + +/* + * use this function, if you want to init SSL for the socket currently registered in the zookeeper handler + */ +static int init_ssl_for_handler(zhandle_t *zh) +{ + int rc = init_ssl_for_socket(zh->fd, zh, 1); + if (rc == ZOK) { + // (SUCCESS) Now mark the ZOO_CONNECTING_STATE so that + // prime_connection() happen. + // prime_connection() only happens in ZOO_CONNECTING_STATE + zh->state = ZOO_CONNECTING_STATE; + } + return rc; +} + +/* + * use this function, if you want to init SSL for a socket, pointing to a different server address than the one + * currently registered in the zookeeper handler (e.g. ping other servers when you are connected to a read-only one) + */ +static int init_ssl_for_socket(zsock_t *fd, zhandle_t *zh, int fail_on_error) { + + SSL_CTX **ctx; + + if (!fd->ssl_sock) { + const SSL_METHOD *method; + +#if OPENSSL_VERSION_NUMBER < 0x10100000L + OpenSSL_add_all_algorithms(); + ERR_load_BIO_strings(); + ERR_load_crypto_strings(); + SSL_load_error_strings(); + SSL_library_init(); + method = SSLv23_client_method(); +#else + OPENSSL_init_ssl(OPENSSL_INIT_LOAD_SSL_STRINGS | OPENSSL_INIT_LOAD_CRYPTO_STRINGS, NULL); + method = TLS_client_method(); +#endif + if (FIPS_mode() == 0) { + LOG_INFO(LOGCALLBACK(zh), "FIPS mode is OFF "); + } else { + LOG_INFO(LOGCALLBACK(zh), "FIPS mode is ON "); + } + fd->ssl_ctx = SSL_CTX_new(method); + ctx = &fd->ssl_ctx; + + SSL_CTX_set_verify(*ctx, SSL_VERIFY_PEER | SSL_VERIFY_FAIL_IF_NO_PEER_CERT, 0); + /*SERVER CA FILE*/ + if (SSL_CTX_load_verify_locations(*ctx, fd->cert->ca, 0) != 1) { + SSL_CTX_free(*ctx); + LOG_ERROR(LOGCALLBACK(zh), "Failed to load CA file %s", fd->cert->ca); + errno = EINVAL; + return ZBADARGUMENTS; + } + if (SSL_CTX_set_default_verify_paths(*ctx) != 1) { + SSL_CTX_free(*ctx); + LOG_ERROR(LOGCALLBACK(zh), "Call to SSL_CTX_set_default_verify_paths failed"); + errno = EINVAL; + return ZBADARGUMENTS; + } + /*CLIENT CA FILE (With Certificate Chain)*/ + if (SSL_CTX_use_certificate_chain_file(*ctx, fd->cert->cert) != 1) { + SSL_CTX_free(*ctx); + LOG_ERROR(LOGCALLBACK(zh), "Failed to load client certificate chain from %s", fd->cert->cert); + errno = EINVAL; + return ZBADARGUMENTS; + } + /*CLIENT PRIVATE KEY*/ + SSL_CTX_set_default_passwd_cb_userdata(*ctx, fd->cert->passwd); + if (SSL_CTX_use_PrivateKey_file(*ctx, fd->cert->key, SSL_FILETYPE_PEM) != 1) { + SSL_CTX_free(*ctx); + LOG_ERROR(LOGCALLBACK(zh), "Failed to load client private key from %s", fd->cert->key); + errno = EINVAL; + return ZBADARGUMENTS; + } + /*CHECK*/ + if (SSL_CTX_check_private_key(*ctx) != 1) { + SSL_CTX_free(*ctx); + LOG_ERROR(LOGCALLBACK(zh), "SSL_CTX_check_private_key failed"); + errno = EINVAL; + return ZBADARGUMENTS; + } + /*MULTIPLE HANDSHAKE*/ + SSL_CTX_set_mode(*ctx, SSL_MODE_AUTO_RETRY); + + fd->ssl_sock = SSL_new(*ctx); + if (fd->ssl_sock == NULL) { + if (fail_on_error) { + return handle_socket_error_msg(zh,__LINE__,ZSSLCONNECTIONERROR, "error creating ssl context"); + } else { + LOG_ERROR(LOGCALLBACK(zh), "error creating ssl context"); + return ZSSLCONNECTIONERROR; + } + + } + SSL_set_fd(fd->ssl_sock, fd->sock); + } + while(1) { + int rc; + int sock = fd->sock; + struct timeval tv; + fd_set s_rfds, s_wfds; + tv.tv_sec = 1; + tv.tv_usec = 0; + FD_ZERO(&s_rfds); + FD_ZERO(&s_wfds); + rc = SSL_connect(fd->ssl_sock); + if (rc == 1) { + return ZOK; + } else { + rc = SSL_get_error(fd->ssl_sock, rc); + if (rc == SSL_ERROR_WANT_READ) { + FD_SET(sock, &s_rfds); + FD_CLR(sock, &s_wfds); + } else if (rc == SSL_ERROR_WANT_WRITE) { + FD_SET(sock, &s_wfds); + FD_CLR(sock, &s_rfds); + } else { + if (fail_on_error) { + return handle_socket_error_msg(zh,__LINE__,ZSSLCONNECTIONERROR, "error in ssl connect"); + } else { + LOG_ERROR(LOGCALLBACK(zh), "error in ssl connect"); + return ZSSLCONNECTIONERROR; + } + } + rc = select(sock + 1, &s_rfds, &s_wfds, NULL, &tv); + if (rc == -1) { + if (fail_on_error) { + return handle_socket_error_msg(zh,__LINE__,ZSSLCONNECTIONERROR, "error in ssl connect (after select)"); + } else { + LOG_ERROR(LOGCALLBACK(zh), "error in ssl connect (after select)"); + return ZSSLCONNECTIONERROR; + } + } + } + } +} + + +#endif + static int check_events(zhandle_t *zh, int events) { - if (zh->fd == -1) + if (zh->fd->sock == -1) return ZINVALIDSTATE; + +#ifdef HAVE_OPENSSL_H + if ((events&ZOOKEEPER_WRITE) && (zh->state == ZOO_SSL_CONNECTING_STATE) && zh->fd->cert != NULL) { + int rc, error; + socklen_t len = sizeof(error); + rc = getsockopt(zh->fd->sock, SOL_SOCKET, SO_ERROR, &error, &len); + /* the description in section 16.4 "Non-blocking connect" + * in UNIX Network Programming vol 1, 3rd edition, points out + * that sometimes the error is in errno and sometimes in error */ + if (rc < 0 || error) { + if (rc == 0) + errno = error; + return handle_socket_error_msg(zh, __LINE__,ZCONNECTIONLOSS, + "server refused to accept the client"); + } + // We do SSL_connect() here + if (init_ssl_for_handler(zh) != ZOK) { + return ZSSLCONNECTIONERROR; + } + } +#endif + if ((events&ZOOKEEPER_WRITE)&&(zh->state == ZOO_CONNECTING_STATE)) { int rc, error; socklen_t len = sizeof(error); - rc = getsockopt(zh->fd, SOL_SOCKET, SO_ERROR, &error, &len); + rc = getsockopt(zh->fd->sock, SOL_SOCKET, SO_ERROR, &error, &len); /* the description in section 16.4 "Non-blocking connect" * in UNIX Network Programming vol 1, 3rd edition, points out * that sometimes the error is in errno and sometimes in error */ @@ -2475,6 +2737,7 @@ static int check_events(zhandle_t *zh, int events) LOG_INFO(LOGCALLBACK(zh), "initiated connection to server %s", format_endpoint_info(&zh->addr_cur)); return ZOK; } + if (zh->to_send.head && (events&ZOOKEEPER_WRITE)) { /* make the flush call non-blocking by specifying a 0 timeout */ int rc=flush_send_queue(zh,0); @@ -2811,7 +3074,7 @@ static void isSocketReadable(zhandle_t* zh) { #ifndef _WIN32 struct pollfd fds; - fds.fd = zh->fd; + fds.fd = zh->fd->sock; fds.events = POLLIN; if (poll(&fds,1,0)<=0) { // socket not readable -- no more responses to process @@ -3285,6 +3548,7 @@ int zookeeper_close(zhandle_t *zh) finish: destroy(zh); adaptor_destroy(zh); + free(zh->fd); free(zh); #ifdef _WIN32 Win32WSACleanup(); @@ -4346,11 +4610,11 @@ int flush_send_queue(zhandle_t*zh, int timeout) #ifdef _WIN32 wait = get_timeval(timeout-elapsed); FD_ZERO(&pollSet); - FD_SET(zh->fd, &pollSet); + FD_SET(zh->fd->sock, &pollSet); // Poll the socket - rc = select((int)(zh->fd)+1, NULL, &pollSet, NULL, &wait); + rc = select((int)(zh->fd->sock)+1, NULL, &pollSet, NULL, &wait); #else - fds.fd = zh->fd; + fds.fd = zh->fd->sock; fds.events = POLLOUT; fds.revents = 0; rc = poll(&fds, 1, timeout-elapsed); @@ -4362,7 +4626,7 @@ int flush_send_queue(zhandle_t*zh, int timeout) } } - rc = send_buffer(zh->fd, zh->to_send.head); + rc = send_buffer(zh, zh->to_send.head); if(rc==0 && timeout==0){ /* send_buffer would block while sending this buffer */ rc = ZOK; diff --git a/zookeeper-client/zookeeper-client-c/ssl/gencerts.sh b/zookeeper-client/zookeeper-client-c/ssl/gencerts.sh new file mode 100755 index 00000000000..528af1f98bf --- /dev/null +++ b/zookeeper-client/zookeeper-client-c/ssl/gencerts.sh @@ -0,0 +1,123 @@ +#!/usr/bin/env bash + +# 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. + +# +# This script cleans up old transaction logs and snapshots +# + +# +# If this scripted is run out of /usr/bin or some other system bin directory +# it should be linked to and not copied. Things like java jar files are found +# relative to the canonical path of this script. +# + +# use local fully qualified domain name in the certificates, or fall back +# to zookeeper.apache.org if no domain name is set or the `hostname` command fails +FQDN=`hostname -f` +FQDN=${FQDN:-"zookeeper.apache.org"} + +# Generate the root key +openssl genrsa -out rootkey.pem 2048 + +#Generate the root Cert +openssl req -x509 -new -key rootkey.pem -out root.crt -config <( +cat <<-EOF +[ req ] +default_bits = 2048 +prompt = no +default_md = sha256 +distinguished_name = dn + +[ dn ] +C = US +ST = California +L = San Francisco +O = ZooKeeper +emailAddress = dev@$FQDN +CN = $FQDN +EOF +) + +#Generate Client Key +openssl genrsa -out clientkey.pem 2048 + +#Generate Client Cert +openssl req -new -key clientkey.pem -out client.csr -config <( +cat <<-EOF +[ req ] +default_bits = 2048 +prompt = no +default_md = sha256 +distinguished_name = dn + +[ dn ] +C = US +ST = California +L = San Francisco +O = ZooKeeper +emailAddress = dev@$FQDN +CN = $FQDN +EOF +) +openssl x509 -req -in client.csr -CA root.crt -CAkey rootkey.pem -CAcreateserial -days 3650 -out client.crt + +#Export in pkcs12 format +openssl pkcs12 -export -in client.crt -inkey clientkey.pem -out client.pkcs12 -password pass:password + +# Import Keystore in JKS +keytool -importkeystore -srckeystore client.pkcs12 -destkeystore client.jks -srcstoretype pkcs12 -srcstorepass password -deststorepass password + +############################################################ + +#Generate Server key +openssl genrsa -out serverkey.pem 2048 + +#Generate Server Cert +openssl req -new -key serverkey.pem -out server.csr -config <( +cat <<-EOF +[ req ] +default_bits = 2048 +prompt = no +default_md = sha256 +distinguished_name = dn + +[ dn ] +C = US +ST = California +L = San Francisco +O = ZooKeeper +emailAddress = dev@$FQDN +CN = $FQDN +EOF +) +openssl x509 -req -in server.csr -CA root.crt -CAkey rootkey.pem -CAcreateserial -days 3650 -out server.crt + +#Export in pkcs12 format +openssl pkcs12 -export -in server.crt -inkey serverkey.pem -out server.pkcs12 -password pass:password + +# Import Keystore in JKS +keytool -importkeystore -srckeystore server.pkcs12 -destkeystore server.jks -srcstoretype pkcs12 -srcstorepass password -deststorepass password + + +keytool -importcert -keystore server.jks -file root.crt -storepass password -noprompt + +keytool -importcert -alias ca -file root.crt -keystore clienttrust.jks -storepass password -noprompt + +keytool -importcert -alias clientcert -file client.crt -keystore clienttrust.jks -storepass password -noprompt + +keytool -importcert -alias ca -file root.crt -keystore servertrust.jks -storepass password -noprompt +keytool -importcert -alias servercert -file server.crt -keystore servertrust.jks -storepass password -noprompt diff --git a/zookeeper-client/zookeeper-client-c/tests/TestClient.cc b/zookeeper-client/zookeeper-client-c/tests/TestClient.cc index 07f9f568f83..a7d055fbe47 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestClient.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestClient.cc @@ -209,6 +209,9 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture CPPUNIT_TEST(testNullData); #ifdef ZOO_IPV6_ENABLED CPPUNIT_TEST(testIPV6); +#endif +#ifdef HAVE_OPENSSL_H + CPPUNIT_TEST(testSSL); #endif CPPUNIT_TEST(testCreate); CPPUNIT_TEST(testCreateContainer); @@ -267,7 +270,16 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture sleep(1); return zk; } - + +#ifdef HAVE_OPENSSL_H + zhandle_t *createSSLClient(const char *hp, const char *cert, watchctx_t *ctx) { + zhandle_t *zk = zookeeper_init_ssl(hp, cert, watcher, 30000, 0, ctx, 0); + ctx->zh = zk; + sleep(1); + return zk; + } +#endif + zhandle_t *createchClient(watchctx_t *ctx, const char* chroot) { zhandle_t *zk = zookeeper_init(chroot, watcher, 10000, 0, ctx, 0); ctx->zh = zk; @@ -363,7 +375,7 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture sleep(1); zh->io_count = 0; //close socket - close(zh->fd); + close_zsock(zh->fd); sleep(1); //Check that doIo isn't spinning CPPUNIT_ASSERT(zh->io_count < 2); @@ -789,6 +801,19 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT_EQUAL((int) ZOK, rc); } +#ifdef HAVE_OPENSSL_H + void testSSL() { + watchctx_t ctx; + zoo_set_debug_level(ZOO_LOG_LEVEL_DEBUG); + zhandle_t *zk = createSSLClient("127.0.0.1:22281", "/tmp/certs/server.crt,/tmp/certs/client.crt,/tmp/certs/clientkey.pem,password", &ctx); + CPPUNIT_ASSERT(zk); + int rc = 0; + rc = zoo_create(zk, "/ssl", NULL, -1, + &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0); + CPPUNIT_ASSERT_EQUAL((int) ZOK, rc); + } +#endif + void testNullData() { watchctx_t ctx; zhandle_t *zk = createClient(&ctx); @@ -1380,7 +1405,7 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture } void testRemoveWatchers() { - char *path = "/something"; + const char *path = "/something"; char buf[1024]; int blen = sizeof(buf); int rc; diff --git a/zookeeper-client/zookeeper-client-c/tests/TestClientRetry.cc b/zookeeper-client/zookeeper-client-c/tests/TestClientRetry.cc index c6536dc91e5..c8e9e158483 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestClientRetry.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestClientRetry.cc @@ -93,7 +93,7 @@ class Zookeeper_clientretry : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT(system(cmd) == 0); /* we are testing that if max cnxns is exceeded the server does the right thing */ - sprintf(cmd, "export ZKMAXCNXNS=1;%s startClean %s", ZKSERVER_CMD, getHostPorts()); + sprintf(cmd, "ZKMAXCNXNS=1 %s startClean %s", ZKSERVER_CMD, getHostPorts()); CPPUNIT_ASSERT(system(cmd) == 0); struct sigaction act; diff --git a/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc b/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc index ec1096df765..226e4708202 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc @@ -52,7 +52,6 @@ using namespace std; int interest; int events; struct timeval tv; - int rc; time_t expires = time(0) + seconds; time_t timeLeft = seconds; fd_set rfds, wfds, efds; @@ -80,7 +79,7 @@ using namespace std; if (tv.tv_sec > timeLeft) { tv.tv_sec = timeLeft; } - rc = select(fd+1, &rfds, &wfds, &efds, &tv); + select(fd+1, &rfds, &wfds, &efds, &tv); timeLeft = expires - time(0); events = 0; if (FD_ISSET(fd, &rfds)) { diff --git a/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc b/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc index d73f1896cfe..0d4d579f444 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc @@ -31,6 +31,9 @@ class Zookeeper_readOnly : public CPPUNIT_NS::TestFixture { CPPUNIT_TEST_SUITE(Zookeeper_readOnly); CPPUNIT_TEST(testReadOnly); +#ifdef HAVE_OPENSSL_H + CPPUNIT_TEST(testReadOnlyWithSSL); +#endif CPPUNIT_TEST_SUITE_END(); static void watcher(zhandle_t* zh, int type, int state, @@ -67,11 +70,12 @@ class Zookeeper_readOnly : public CPPUNIT_NS::TestFixture { void setUp() { zoo_set_log_stream(logfile); + zoo_set_debug_level(ZOO_LOG_LEVEL_DEBUG); } void startReadOnly() { char cmd[1024]; - sprintf(cmd, "%s startReadOnly", ZKSERVER_CMD); + sprintf(cmd, "%s startCleanReadOnly", ZKSERVER_CMD); CPPUNIT_ASSERT(system(cmd) == 0); } @@ -81,29 +85,68 @@ class Zookeeper_readOnly : public CPPUNIT_NS::TestFixture { CPPUNIT_ASSERT(system(cmd) == 0); } - void testReadOnly() { - startReadOnly(); - watchctx_t watch; - zhandle_t* zh = zookeeper_init("localhost:22181", - watcher, - 10000, - NULL, - &watch, - ZOO_READONLY); - watch.zh = zh; + zhandle_t* connectReadOnly(const char *address, watchctx_t *watch) { + zhandle_t* zh = zookeeper_init(address, watcher, 10000, NULL, watch, ZOO_READONLY); + watch->zh = zh; CPPUNIT_ASSERT(zh != 0); sleep(1); + return zh; + } + + void assertCanRead(zhandle_t* zh, const char *znode_path) { int len = 1024; char buf[len]; - int res = zoo_get(zh, "/", 0, buf, &len, 0); + int res = zoo_get(zh, znode_path, 0, buf, &len, 0); CPPUNIT_ASSERT_EQUAL((int)ZOK, res); + } + void assertCanNotWrite(zhandle_t* zh, const char *znode_path) { char path[1024]; - res = zoo_create(zh, "/test", buf, 10, &ZOO_OPEN_ACL_UNSAFE, 0, path, - 512); + char buf[1024]; + int res = zoo_create(zh, znode_path, buf, 10, &ZOO_OPEN_ACL_UNSAFE, 0, path, 512); CPPUNIT_ASSERT_EQUAL((int)ZNOTREADONLY, res); + } + + void testReadOnly() + { + startReadOnly(); + + watchctx_t watch; + zhandle_t* zh = connectReadOnly("localhost:22181", &watch); + + assertCanRead(zh, "/"); + + assertCanNotWrite(zh, "/test"); + + stopPeer(); + } + +#ifdef HAVE_OPENSSL_H + + zhandle_t* connectReadOnlySSL(const char *address, const char *certs, watchctx_t *watch) { + zhandle_t* zh = zookeeper_init_ssl(address, certs, watcher, 10000, NULL, watch, ZOO_READONLY); + watch->zh = zh; + CPPUNIT_ASSERT(zh != 0); + sleep(1); + return zh; + } + + void testReadOnlyWithSSL() { + startReadOnly(); + + watchctx_t watch; + zhandle_t* zh = connectReadOnlySSL("localhost:22281", + "/tmp/certs/server.crt,/tmp/certs/client.crt,/tmp/certs/clientkey.pem,password", + &watch); + + assertCanRead(zh, "/"); + + assertCanNotWrite(zh, "/testSSL"); + stopPeer(); } +#endif + }; CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_readOnly); diff --git a/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc b/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc index 97940b6a76c..317ffcddfed 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc @@ -265,7 +265,7 @@ class Zookeeper_reconfig : public CPPUNIT_NS::TestFixture void tearDown() { - for (int i = 0; i < clients.size(); i++) + for (unsigned int i = 0; i < clients.size(); i++) { clients.at(i).close(); } @@ -316,7 +316,7 @@ class Zookeeper_reconfig : public CPPUNIT_NS::TestFixture stringstream ss; - for (int i = start; i >= stop; i--, octet--) + for (uint32_t i = start; i >= stop; i--, octet--) { ss << "10.10.10." << octet << ":" << portOffset + octet; @@ -573,8 +573,6 @@ class Zookeeper_reconfig : public CPPUNIT_NS::TestFixture { zoo_deterministic_conn_order(0); - int rc = ZOK; - uint32_t numServers = 9; const string initial_hosts = createHostList(numServers); // 10.10.10.9:2009...10.10.10.1:2001 @@ -584,7 +582,7 @@ class Zookeeper_reconfig : public CPPUNIT_NS::TestFixture numClientsPerHost.at(client.getServerPort() - portOffset - 1)++; } - for (int i = 0; i < numServers; i++) { + for (uint32_t i = 0; i < numServers; i++) { CPPUNIT_ASSERT(numClientsPerHost.at(i) <= upperboundClientsPerServer(numClients, numServers)); CPPUNIT_ASSERT(numClientsPerHost.at(i) >= lowerboundClientsPerServer(numClients, numServers)); numClientsPerHost.at(i) = 0; // prepare for next test diff --git a/zookeeper-client/zookeeper-client-c/tests/TestServerRequireClientSASLAuth.cc b/zookeeper-client/zookeeper-client-c/tests/TestServerRequireClientSASLAuth.cc index 2c5290ded55..294388874f2 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestServerRequireClientSASLAuth.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestServerRequireClientSASLAuth.cc @@ -104,6 +104,6 @@ class Zookeeper_serverRequireClientSASL : public CPPUNIT_NS::TestFixture { } }; -const char Zookeeper_serverRequireClientSASL::hostPorts[] = "127.0.0.1:23456"; +const char Zookeeper_serverRequireClientSASL::hostPorts[] = "127.0.0.1:22181"; CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_serverRequireClientSASL); diff --git a/zookeeper-client/zookeeper-client-c/tests/TestZookeeperClose.cc b/zookeeper-client/zookeeper-client-c/tests/TestZookeeperClose.cc index 6d704364f98..f20910f506f 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestZookeeperClose.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestZookeeperClose.cc @@ -110,7 +110,7 @@ class Zookeeper_close : public CPPUNIT_NS::TestFixture zh=zookeeper_init("localhost:2121",watcher,10000,0,0,0); CPPUNIT_ASSERT(zh!=0); // simulate connected state - zh->fd=ZookeeperServer::FD; + zh->fd->sock=ZookeeperServer::FD; zh->state=ZOO_CONNECTED_STATE; Mock_flush_send_queue zkMock; // do not actually free the memory while in zookeeper_close() diff --git a/zookeeper-client/zookeeper-client-c/tests/TestZookeeperInit.cc b/zookeeper-client/zookeeper-client-c/tests/TestZookeeperInit.cc index 6f43e050905..61da41dd547 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestZookeeperInit.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestZookeeperInit.cc @@ -102,7 +102,7 @@ class Zookeeper_init : public CPPUNIT_NS::TestFixture &cid,(void*)1,0); CPPUNIT_ASSERT(zh != NULL); - CPPUNIT_ASSERT(zh->fd == -1); + CPPUNIT_ASSERT(zh->fd->sock == -1); CPPUNIT_ASSERT(zh->hostname != NULL); CPPUNIT_ASSERT_EQUAL(EXPECTED_ADDRS_COUNT,zh->addrs.count); CPPUNIT_ASSERT_EQUAL(EXPECTED_HOST,string(zh->hostname)); @@ -143,7 +143,7 @@ class Zookeeper_init : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT(zh!=0); CPPUNIT_ASSERT_EQUAL(EXPECTED_ADDRS_COUNT,zh->addrs.count); - for(int i=0;iaddrs.count;i++){ + for(unsigned int i=0;iaddrs.count;i++){ sockaddr_in* addr=(struct sockaddr_in*)&zh->addrs.data[i]; CPPUNIT_ASSERT(memcmp(EXPECTED_IPS[i],&addr->sin_addr,sizeof(addr->sin_addr))==0); CPPUNIT_ASSERT_EQUAL(2121,(int)ntohs(addr->sin_port)); @@ -161,7 +161,7 @@ class Zookeeper_init : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT(zh!=0); CPPUNIT_ASSERT_EQUAL(EXPECTED_ADDRS_COUNT,zh->addrs.count); - for(int i=0;iaddrs.count;i++){ + for(unsigned int i=0;iaddrs.count;i++){ sockaddr_in* addr=(struct sockaddr_in*)&zh->addrs.data[i]; CPPUNIT_ASSERT(memcmp(EXPECTED_IPS[i],&addr->sin_addr,sizeof(addr->sin_addr))==0); if(i<1) @@ -182,7 +182,7 @@ class Zookeeper_init : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT(zh!=0); CPPUNIT_ASSERT_EQUAL(EXPECTED_ADDRS_COUNT,zh->addrs.count); - for(int i=0;iaddrs.count;i++){ + for(unsigned int i=0;iaddrs.count;i++){ sockaddr_in* addr=(struct sockaddr_in*)&zh->addrs.data[i]; CPPUNIT_ASSERT(memcmp(EXPECTED_IPS[i],&addr->sin_addr,sizeof(addr->sin_addr))==0); if(i<1) @@ -289,7 +289,7 @@ class Zookeeper_init : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT_EQUAL(EXPECTED_ADDR_COUNT,zh->addrs.count); const string EXPECTED_SEQ("3210"); char ACTUAL_SEQ[EXPECTED_ADDR_COUNT+1]; ACTUAL_SEQ[EXPECTED_ADDR_COUNT]=0; - for(int i=0;iaddrs.count;i++){ + for(unsigned int i=0;iaddrs.count;i++){ sockaddr_in* addr=(struct sockaddr_in*)&zh->addrs.data[i]; // match the first byte of the EXPECTED and of the actual address ACTUAL_SEQ[i]=((char*)&addr->sin_addr)[0]+'0'; diff --git a/zookeeper-client/zookeeper-client-c/tests/WatchUtil.h b/zookeeper-client/zookeeper-client-c/tests/WatchUtil.h index 035f26ded79..8bc4dbc3e3a 100644 --- a/zookeeper-client/zookeeper-client-c/tests/WatchUtil.h +++ b/zookeeper-client/zookeeper-client-c/tests/WatchUtil.h @@ -42,7 +42,6 @@ using namespace Util; int interest; int events; struct timeval tv; - int rc; time_t expires = time(0) + seconds; time_t timeLeft = seconds; fd_set rfds, wfds, efds; @@ -70,7 +69,7 @@ using namespace Util; if (tv.tv_sec > timeLeft) { tv.tv_sec = timeLeft; } - rc = select(fd+1, &rfds, &wfds, &efds, &tv); + select(fd+1, &rfds, &wfds, &efds, &tv); timeLeft = expires - time(0); events = 0; if (FD_ISSET(fd, &rfds)) { diff --git a/zookeeper-client/zookeeper-client-c/tests/ZKMocks.cc b/zookeeper-client/zookeeper-client-c/tests/ZKMocks.cc index 1310ab9d4b8..d26c295f24d 100644 --- a/zookeeper-client/zookeeper-client-c/tests/ZKMocks.cc +++ b/zookeeper-client/zookeeper-client-c/tests/ZKMocks.cc @@ -33,7 +33,7 @@ TestClientId testClientId; const char* TestClientId::PASSWD="1234567890123456"; HandshakeRequest* HandshakeRequest::parse(const std::string& buf) { - auto_ptr req(new HandshakeRequest); + unique_ptr req(new HandshakeRequest); memcpy(&req->protocolVersion,buf.data(), sizeof(req->protocolVersion)); req->protocolVersion = htonl(req->protocolVersion); @@ -480,7 +480,7 @@ void ZookeeperServer::onMessageReceived(const RequestHeader& rh, iarchive* ia){ void ZookeeperServer::notifyBufferSent(const std::string& buffer){ if(HandshakeRequest::isValid(buffer)){ // could be a connect request - auto_ptr req(HandshakeRequest::parse(buffer)); + unique_ptr req(HandshakeRequest::parse(buffer)); if(req.get()!=0){ // handle the handshake int64_t sessId=sessionExpired?req->sessionId+1:req->sessionId; @@ -528,7 +528,7 @@ void forceConnected(zhandle_t* zh){ zh->state=ZOO_CONNECTED_STATE; // Simulate we're connected to the first host in our host list - zh->fd=ZookeeperServer::FD; + zh->fd->sock=ZookeeperServer::FD; assert(zh->addrs.count > 0); zh->addr_cur = zh->addrs.data[0]; zh->addrs.next++; diff --git a/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.cc b/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.cc index c38e385a15c..db79919c3c7 100644 --- a/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.cc +++ b/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.cc @@ -121,7 +121,7 @@ createConfigFile(std::string config) { confFile << "initLimit=5\n"; confFile << "syncLimit=2\n"; confFile << "dataDir=" << getDataDirectory() << "\n"; - for (int i = 0; i < numServers_; i++) { + for (uint32_t i = 0; i < numServers_; i++) { confFile << getServerString(i) << "\n"; } // Append additional config, if any. @@ -177,14 +177,14 @@ getDataDirectory() { std::vector ZooKeeperQuorumServer:: getCluster(uint32_t numServers) { std::vector cluster; - for (int i = 0; i < numServers; i++) { + for (uint32_t i = 0; i < numServers; i++) { cluster.push_back(new ZooKeeperQuorumServer(i, numServers)); } // Wait until all the servers start, and fail if they don't start within 10 // seconds. - for (int i = 0; i < 10; i++) { - int j = 0; + for (uint32_t i = 0; i < 10; i++) { + uint32_t j = 0; for (; j < cluster.size(); j++) { if (cluster[j]->getMode() == "") { // The server hasn't started. @@ -207,14 +207,14 @@ getCluster(uint32_t numServers, ZooKeeperQuorumServer::tConfigPairs configs, std std::pair pair = *iter; config += (pair.first + "=" + pair.second + "\n"); } - for (int i = 0; i < numServers; i++) { + for (uint32_t i = 0; i < numServers; i++) { cluster.push_back(new ZooKeeperQuorumServer(i, numServers, config, env)); } // Wait until all the servers start, and fail if they don't start within 10 // seconds. - for (int i = 0; i < 10; i++) { - int j = 0; + for (uint32_t i = 0; i < 10; i++) { + uint32_t j = 0; for (; j < cluster.size(); j++) { if (cluster[j]->getMode() == "") { // The server hasn't started. diff --git a/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.h b/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.h index 577072ebcc4..18a8d63e27a 100644 --- a/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.h +++ b/zookeeper-client/zookeeper-client-c/tests/ZooKeeperQuorumServer.h @@ -55,10 +55,10 @@ class ZooKeeperQuorumServer { static const uint32_t ELECTION_PORT_BASE = 3000; static const uint32_t CLIENT_PORT_BASE = 4000; - uint32_t numServers_; uint32_t id_; - std::string root_; std::string env_; + uint32_t numServers_; + std::string root_; }; #endif // ZOOKEEPER_QUORUM_SERVER_H diff --git a/zookeeper-client/zookeeper-client-c/tests/quorum.cfg b/zookeeper-client/zookeeper-client-c/tests/quorum.cfg deleted file mode 100644 index cb0aa81c856..00000000000 --- a/zookeeper-client/zookeeper-client-c/tests/quorum.cfg +++ /dev/null @@ -1,8 +0,0 @@ -tickTime=500 -initLimit=10 -syncLimit=5 -dataDir=TMPDIR/zkdata -clientPort=22181 -server.1=localhost:22881:33881 -server.2=localhost:22882:33882 -server.3=localhost:22883:33883 diff --git a/zookeeper-client/zookeeper-client-c/tests/zkServer.sh b/zookeeper-client/zookeeper-client-c/tests/zkServer.sh index ebc3df42ba0..432786ce786 100755 --- a/zookeeper-client/zookeeper-client-c/tests/zkServer.sh +++ b/zookeeper-client/zookeeper-client-c/tests/zkServer.sh @@ -17,28 +17,31 @@ # limitations under the License. # This is the port where zookeeper server runs on. -ZOOPORT=22181 +ZOOPORT=${ZOOPORT:-"22181"} + +# Some tests are setting the maxClientConnections. When it is not set, we fallback to default 100 +ZKMAXCNXNS=${ZKMAXCNXNS:-"100"} + +EXTRA_JVM_ARGS=${EXTRA_JVM_ARGS:-""} if [ "x$1" == "x" ] then - echo "USAGE: $0 startClean|start|startReadOnly|startRequireSASLAuth|stop hostPorts" + echo "USAGE: $0 startClean|start|startCleanReadOnly|startRequireSASLAuth|stop" exit 2 fi + + + +# ===== +# ===== cleanup old executions +# ===== + case "`uname`" in CYGWIN*) cygwin=true ;; *) cygwin=false ;; esac -if [ "x$1" == "xstartClean" ] -then - if [ "x${base_dir}" == "x" ] - then - rm -rf /tmp/zkdata - else - rm -rf "${base_dir}/build/tmp" - fi -fi if $cygwin then @@ -75,6 +78,12 @@ then fi fi + + +# ===== +# ===== build classpath +# ===== + if [ "x${base_dir}" == "x" ] then zk_base="../../../" @@ -108,46 +117,115 @@ then CLASSPATH=`cygpath -wp "$CLASSPATH"` fi -PROPERTIES="-Dzookeeper.extendedTypesEnabled=true -Dznode.container.checkIntervalMs=100" + + +# ===== +# ===== initialize JVM arguments +# ===== + +PROPERTIES="$EXTRA_JVM_ARGS -Dzookeeper.extendedTypesEnabled=true -Dznode.container.checkIntervalMs=100" +if [ "x$1" == "xstartRequireSASLAuth" ] +then + PROPERTIES="-Dzookeeper.sessionRequireClientSASLAuth=true $PROPERTIES" +fi +if [ "x$1" == "xstartCleanReadOnly" ] +then + PROPERTIES="-Dreadonlymode.enabled=true $PROPERTIES" +fi + + + +# ===== +# ===== initialize data and test directories +# ===== + +if [ "x${base_dir}" == "x" ] +then + tmp_dir="/tmp" + tests_dir="tests" +else + tmp_dir="${base_dir}/build/tmp" + tests_dir=${base_dir}/zookeeper-client/zookeeper-client-c/tests +fi + + + + +# ===== +# ===== start the ZooKeeper server +# ===== case $1 in -start|startClean) - if [ "x${base_dir}" == "x" ] +start|startClean|startRequireSASLAuth|startCleanReadOnly) + + if [ "x$1" == "xstartClean" ] || [ "x$1" == "xstartCleanReadOnly" ] + then + rm -rf "${tmp_dir}/zkdata" + fi + mkdir -p "${tmp_dir}/zkdata" + + + # ===== initialize certificates + certs_dir="/tmp/certs" + rm -rf "${certs_dir}" + mkdir -p "${certs_dir}" + cp ${tests_dir}/../ssl/gencerts.sh "${certs_dir}/" > /dev/null + cd ${certs_dir} > /dev/null + ./gencerts.sh > ./gencerts.stdout 2> ./gencerts.stderr + cd - > /dev/null + + + # ===== prepare the configs + sed "s#TMPDIR#${tmp_dir}#g;s#CERTDIR#${certs_dir}#g;s#MAXCLIENTCONNECTIONS#${ZKMAXCNXNS}#g;s#CLIENTPORT#${ZOOPORT}#g" ${tests_dir}/zoo.cfg > "${tmp_dir}/zoo.cfg" + if [ "x$1" == "xstartCleanReadOnly" ] then - mkdir -p /tmp/zkdata - java -cp "$CLASSPATH" $PROPERTIES org.apache.zookeeper.server.ZooKeeperServerMain $ZOOPORT /tmp/zkdata 3000 $ZKMAXCNXNS &> /tmp/zk.log & - pid=$! - echo -n $! > /tmp/zk.pid + # we can put the new server to read-only mode by starting only a single instance of a three node server + echo "server.1=localhost:22881:33881" >> ${tmp_dir}/zoo.cfg + echo "server.2=localhost:22882:33882" >> ${tmp_dir}/zoo.cfg + echo "server.3=localhost:22883:33883" >> ${tmp_dir}/zoo.cfg + echo "1" > ${tmp_dir}/zkdata/myid + main_class="org.apache.zookeeper.server.quorum.QuorumPeerMain" else - mkdir -p "${base_dir}/build/tmp/zkdata" - java -cp "$CLASSPATH" $PROPERTIES org.apache.zookeeper.server.ZooKeeperServerMain $ZOOPORT "${base_dir}/build/tmp/zkdata" 3000 $ZKMAXCNXNS &> "${base_dir}/build/tmp/zk.log" & - pid=$! - echo -n $pid > "${base_dir}/build/tmp/zk.pid" + main_class="org.apache.zookeeper.server.ZooKeeperServerMain" fi - # wait max 120 seconds for server to be ready to server clients - # this handles testing on slow hosts - success=false - for i in {1..120} - do - if ps -p $pid > /dev/null - then - java -cp "$CLASSPATH" $PROPERTIES org.apache.zookeeper.ZooKeeperMain -server localhost:$ZOOPORT ls / > /dev/null 2>&1 - if [ $? -ne 0 ] + + # ===== start the server + java -cp "$CLASSPATH" $PROPERTIES ${main_class} ${tmp_dir}/zoo.cfg &> "${tmp_dir}/zk.log" & + pid=$! + echo -n $! > /tmp/zk.pid + + + # ===== wait for the server to start + if [ "x$1" == "xstartRequireSASLAuth" ] || [ "x$1" == "xstartCleanReadOnly" ] + then + # ===== in these cases we can not connect simply with the java client, so we are just waiting... + sleep 4 + success=true + else + # ===== wait max 120 seconds for server to be ready to server clients (this handles testing on slow hosts) + success=false + for i in {1..120} + do + if ps -p $pid > /dev/null then - # server not up yet - wait - sleep 1 + java -cp "$CLASSPATH" $PROPERTIES org.apache.zookeeper.ZooKeeperMain -server localhost:$ZOOPORT ls / > /dev/null 2>&1 + if [ $? -ne 0 ] + then + # server not up yet - wait + sleep 1 + else + # server is up and serving client connections + success=true + break + fi else - # server is up and serving client connections - success=true + # server died - exit now + echo -n " ZooKeeper server process failed" break fi - else - # server died - exit now - echo -n " ZooKeeper server process failed" - break - fi - done + done + fi if $success then @@ -158,41 +236,6 @@ start|startClean) echo -n " ZooKeeper server NOT started" fi - ;; -startReadOnly) - if [ "x${base_dir}" == "x" ] - then - echo "this target is for unit tests only" - exit 2 - else - tmpdir="${base_dir}/build/tmp" - mkdir -p "${tmpdir}/zkdata" - rm -f "${tmpdir}/zkdata/myid" && echo 1 > "${tmpdir}/zkdata/myid" - - sed "s#TMPDIR#${tmpdir}#g" ${base_dir}/zookeeper-client/zookeeper-client-c/tests/quorum.cfg > "${tmpdir}/quorum.cfg" - - # force read-only mode - PROPERTIES="$PROPERTIES -Dreadonlymode.enabled=true" - java -cp "$CLASSPATH" $PROPERTIES org.apache.zookeeper.server.quorum.QuorumPeerMain ${tmpdir}/quorum.cfg &> "${tmpdir}/zk.log" & - pid=$! - echo -n $pid > "${base_dir}/build/tmp/zk.pid" - sleep 3 # wait until read-only server is up - fi - - ;; -startRequireSASLAuth) - if [ "x${base_dir}" == "x" ] - then - echo "this target is for unit tests only" - exit 2 - else - mkdir -p "${base_dir}/build/tmp/zkdata" - java -cp "$CLASSPATH" -Dzookeeper.sessionRequireClientSASLAuth=true org.apache.zookeeper.server.ZooKeeperServerMain 23456 "${base_dir}/build/tmp/zkdata" 3000 $ZKMAXCNXNS &> "${base_dir}/build/tmp/zk.log" & - pid=$! - echo -n $pid > "${base_dir}/build/tmp/zk.pid" - sleep 3 # wait until server is up. - fi - ;; stop) # Already killed above diff --git a/zookeeper-client/zookeeper-client-c/tests/zoo.cfg b/zookeeper-client/zookeeper-client-c/tests/zoo.cfg new file mode 100644 index 00000000000..b8e0b2a661f --- /dev/null +++ b/zookeeper-client/zookeeper-client-c/tests/zoo.cfg @@ -0,0 +1,14 @@ +tickTime=500 +initLimit=10 +syncLimit=5 +dataDir=TMPDIR/zkdata +maxClientCnxns=MAXCLIENTCONNECTIONS + +clientPort=CLIENTPORT +secureClientPort=22281 +serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory +ssl.keyStore.location=CERTDIR/server.jks +ssl.keyStore.password=password +ssl.trustStore.location=CERTDIR/servertrust.jks +ssl.trustStore.password=password + diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index ab48d0360c2..08a054e129d 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1004,6 +1004,14 @@ property, when available, is noted below. **New in 3.6.0:** The size threshold after which a request is considered a large request. If it is -1, then all requests are considered small, effectively turning off large request throttling. The default is -1. +* *outstandingHandshake.limit* + (Jave system property only: **zookeeper.netty.server.outstandingHandshake.limit**) + The maximum in-flight TLS handshake connections could have in ZooKeeper, + the connections exceed this limit will be rejected before starting handshake. + This setting doesn't limit the max TLS concurrency, but helps avoid herd + effect due to TLS handshake timeout when there are too many in-flight TLS + handshakes. Set it to something like 250 is good enough to avoid herd effect. + #### Cluster Options @@ -1624,6 +1632,15 @@ Both subsystems need to have sufficient amount of threads to achieve peak read t minute. This prevents herding during container deletion. Default is "10000". +* *znode.container.maxNeverUsedIntervalMs* : + (Java system property only) + **New in 3.6.0:** The + maximum interval in milliseconds that a container that has never had + any children is retained. Should be long enough for your client to + create the container, do any needed work and then create children. + Default is "0" which is used to indicate that containers + that have never had any children are never deleted. + #### Debug Observability Configurations diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java index 3713646d631..3859382fadd 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java @@ -116,6 +116,16 @@ public class ClientCnxn { */ private static final int SET_WATCHES_MAX_LENGTH = 128 * 1024; + /* predefined xid's values recognized as special by the server */ + // -1 means notification(WATCHER_EVENT) + public static final int NOTIFICATION_XID = -1; + // -2 is the xid for pings + public static final int PING_XID = -2; + // -4 is the xid for AuthPacket + public static final int AUTHPACKET_XID = -4; + // -8 is the xid for setWatch + public static final int SET_WATCHES_XID = -8; + static class AuthData { AuthData(String scheme, byte[] data) { @@ -857,16 +867,14 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException { ReplyHeader replyHdr = new ReplyHeader(); replyHdr.deserialize(bbia, "header"); - if (replyHdr.getXid() == -2) { - // -2 is the xid for pings + if (replyHdr.getXid() == PING_XID) { LOG.debug( "Got ping response for session id: 0x{} after {}ms.", Long.toHexString(sessionId), ((System.nanoTime() - lastPingSentNs) / 1000000)); return; } - if (replyHdr.getXid() == -4) { - // -4 is the xid for AuthPacket + if (replyHdr.getXid() == AUTHPACKET_XID) { if (replyHdr.getErr() == KeeperException.Code.AUTHFAILED.intValue()) { state = States.AUTH_FAILED; eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.AuthFailed, null)); @@ -875,8 +883,7 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException { LOG.debug("Got auth session id: 0x{}", Long.toHexString(sessionId)); return; } - if (replyHdr.getXid() == -1) { - // -1 means notification + if (replyHdr.getXid() == NOTIFICATION_XID) { LOG.debug("Got notification session id: 0x{}", Long.toHexString(sessionId)); WatcherEvent event = new WatcherEvent(); event.deserialize(bbia, "response"); @@ -1048,7 +1055,7 @@ record = new SetWatches2(setWatchesLastZxid, dataWatchesBatch, existWatchesBatch childWatchesBatch, persistentWatchesBatch, persistentRecursiveWatchesBatch); opcode = OpCode.setWatches2; } - RequestHeader header = new RequestHeader(-8, opcode); + RequestHeader header = new RequestHeader(ClientCnxn.SET_WATCHES_XID, opcode); Packet packet = new Packet(header, new ReplyHeader(), record, null, null); outgoingQueue.addFirst(packet); } @@ -1058,7 +1065,7 @@ record = new SetWatches2(setWatchesLastZxid, dataWatchesBatch, existWatchesBatch for (AuthData id : authInfo) { outgoingQueue.addFirst( new Packet( - new RequestHeader(-4, OpCode.auth), + new RequestHeader(ClientCnxn.AUTHPACKET_XID, OpCode.auth), null, new AuthPacket(0, id.scheme, id.data), null, @@ -1088,7 +1095,7 @@ private List prependChroot(List paths) { private void sendPing() { lastPingSentNs = System.nanoTime(); - RequestHeader h = new RequestHeader(-2, OpCode.ping); + RequestHeader h = new RequestHeader(ClientCnxn.PING_XID, OpCode.ping); queuePacket(h, null, null, null, null, null, null, null, null); } @@ -1657,7 +1664,7 @@ public void addAuthInfo(String scheme, byte[] auth) { } authInfo.add(new AuthData(scheme, auth)); queuePacket( - new RequestHeader(-4, OpCode.auth), + new RequestHeader(ClientCnxn.AUTHPACKET_XID, OpCode.auth), null, new AuthPacket(0, scheme, auth), null, diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java b/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java index 181e4638841..4029c60ede3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java @@ -20,6 +20,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.zookeeper.server.ExitCode; +import org.apache.zookeeper.util.ServiceUtils; public class Version implements org.apache.zookeeper.version.Info { @@ -48,7 +49,7 @@ public static void printUsage() { System.out.print("Usage:\tjava -cp ... org.apache.zookeeper.Version " + "[--full | --short | --revision],\n\tPrints --full version " + "info if no arg specified."); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); } /** @@ -68,7 +69,7 @@ public static void main(String[] args) { } if (args.length == 0 || (args.length == 1 && args[0].equals("--full"))) { System.out.println(getFullVersion()); - System.exit(ExitCode.EXECUTION_FINISHED.getValue()); + ServiceUtils.requestSystemExit(ExitCode.EXECUTION_FINISHED.getValue()); } if (args[0].equals("--short")) { System.out.println(getVersion()); @@ -77,7 +78,7 @@ public static void main(String[] args) { } else { printUsage(); } - System.exit(ExitCode.EXECUTION_FINISHED.getValue()); + ServiceUtils.requestSystemExit(ExitCode.EXECUTION_FINISHED.getValue()); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java index 857b16fcc70..734cf7ec0ee 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java @@ -66,6 +66,7 @@ import org.apache.zookeeper.cli.VersionCommand; import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.server.ExitCode; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -353,7 +354,7 @@ void run() throws CliException, IOException, InterruptedException { // Command line args non-null. Run what was passed. processCmd(cl); } - System.exit(exitCode); + ServiceUtils.requestSystemExit(exitCode); } public void executeLine(String line) throws CliException, InterruptedException, IOException { @@ -396,7 +397,7 @@ protected boolean processZKCmd(MyCommandOptions co) throws CliException, IOExcep if (cmd.equals("quit")) { zk.close(); - System.exit(exitCode); + ServiceUtils.requestSystemExit(exitCode); } else if (cmd.equals("redo") && args.length >= 2) { Integer i = Integer.decode(args[1]); if (commandCount <= i || i < 0) { // don't allow redoing this redo diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZKClientConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZKClientConfig.java index 3012a35f983..10c61375d1d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZKClientConfig.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZKClientConfig.java @@ -99,6 +99,7 @@ protected void handleBackwardCompatibility() { * backward compatibility for client specific properties */ setProperty(ZK_SASL_CLIENT_USERNAME, System.getProperty(ZK_SASL_CLIENT_USERNAME)); + setProperty(ZK_SASL_CLIENT_CANONICALIZE_HOSTNAME, System.getProperty(ZK_SASL_CLIENT_CANONICALIZE_HOSTNAME)); setProperty(LOGIN_CONTEXT_NAME_KEY, System.getProperty(LOGIN_CONTEXT_NAME_KEY)); setProperty(ENABLE_CLIENT_SASL_KEY, System.getProperty(ENABLE_CLIENT_SASL_KEY)); setProperty(ZOOKEEPER_SERVER_REALM, System.getProperty(ZOOKEEPER_SERVER_REALM)); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ContainerManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ContainerManager.java index 81a07a6ed00..66b0240b410 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ContainerManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ContainerManager.java @@ -46,6 +46,7 @@ public class ContainerManager { private final RequestProcessor requestProcessor; private final int checkIntervalMs; private final int maxPerMinute; + private final long maxNeverUsedIntervalMs; private final Timer timer; private final AtomicReference task = new AtomicReference(null); @@ -58,13 +59,28 @@ public class ContainerManager { * herding of container deletions */ public ContainerManager(ZKDatabase zkDb, RequestProcessor requestProcessor, int checkIntervalMs, int maxPerMinute) { + this(zkDb, requestProcessor, checkIntervalMs, maxPerMinute, Long.MAX_VALUE); + } + + /** + * @param zkDb the ZK database + * @param requestProcessor request processer - used to inject delete + * container requests + * @param checkIntervalMs how often to check containers in milliseconds + * @param maxPerMinute the max containers to delete per second - avoids + * herding of container deletions + * @param maxNeverUsedIntervalMs the max time in milliseconds that a container that has never had + * any children is retained + */ + public ContainerManager(ZKDatabase zkDb, RequestProcessor requestProcessor, int checkIntervalMs, int maxPerMinute, long maxNeverUsedIntervalMs) { this.zkDb = zkDb; this.requestProcessor = requestProcessor; this.checkIntervalMs = checkIntervalMs; this.maxPerMinute = maxPerMinute; + this.maxNeverUsedIntervalMs = maxNeverUsedIntervalMs; timer = new Timer("ContainerManagerTask", true); - LOG.info("Using checkIntervalMs={} maxPerMinute={}", checkIntervalMs, maxPerMinute); + LOG.info("Using checkIntervalMs={} maxPerMinute={} maxNeverUsedIntervalMs={}", checkIntervalMs, maxPerMinute, maxNeverUsedIntervalMs); } /** @@ -116,7 +132,7 @@ public void checkContainers() throws InterruptedException { Request request = new Request(null, 0, 0, ZooDefs.OpCode.deleteContainer, path, null); try { LOG.info("Attempting to delete candidate container: {}", containerPath); - requestProcessor.processRequest(request); + postDeleteRequest(request); } catch (Exception e) { LOG.error("Could not delete container: {}", containerPath, e); } @@ -129,6 +145,11 @@ public void checkContainers() throws InterruptedException { } } + // VisibleForTesting + protected void postDeleteRequest(Request request) throws RequestProcessor.RequestProcessorException { + requestProcessor.processRequest(request); + } + // VisibleForTesting protected long getMinIntervalMs() { return TimeUnit.MINUTES.toMillis(1) / maxPerMinute; @@ -139,12 +160,26 @@ protected Collection getCandidates() { Set candidates = new HashSet(); for (String containerPath : zkDb.getDataTree().getContainers()) { DataNode node = zkDb.getDataTree().getNode(containerPath); - /* - cversion > 0: keep newly created containers from being deleted - before any children have been added. If you were to create the - container just before a container cleaning period the container - would be immediately be deleted. - */ + if ((node != null) && node.getChildren().isEmpty()) { + /* + cversion > 0: keep newly created containers from being deleted + before any children have been added. If you were to create the + container just before a container cleaning period the container + would be immediately be deleted. + */ + if (node.stat.getCversion() > 0) { + candidates.add(containerPath); + } else { + /* + Users may not want unused containers to live indefinitely. Allow a system + property to be set that sets the max time for a cversion-0 container + to stay before being deleted + */ + if ((maxNeverUsedIntervalMs != 0) && (getElapsed(node) > maxNeverUsedIntervalMs)) { + candidates.add(containerPath); + } + } + } if ((node != null) && (node.stat.getCversion() > 0) && (node.getChildren().isEmpty())) { candidates.add(containerPath); } @@ -155,7 +190,6 @@ protected Collection getCandidates() { Set children = node.getChildren(); if (children.isEmpty()) { if (EphemeralType.get(node.stat.getEphemeralOwner()) == EphemeralType.TTL) { - long elapsed = getElapsed(node); long ttl = EphemeralType.TTL.getValue(node.stat.getEphemeralOwner()); if ((ttl != 0) && (getElapsed(node) > ttl)) { candidates.add(ttlPath); 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 f370d88f400..0dc54580699 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 @@ -78,6 +78,7 @@ import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.Txn; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -301,7 +302,7 @@ public DataTree() { childWatches = WatchManagerFactory.createWatchManager(); } catch (Exception e) { LOG.error("Unexpected exception when creating WatchManager, exiting abnormally", e); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); } } 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 ee758b272da..9ffde55c10c 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 @@ -27,6 +27,7 @@ import java.util.Set; import org.apache.commons.lang.StringUtils; import org.apache.jute.Record; +import org.apache.zookeeper.ClientCnxn; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.KeeperException.SessionMovedException; @@ -200,7 +201,7 @@ public void processRequest(Request request) { lastOp = "PING"; updateStats(request, lastOp, lastZxid); - cnxn.sendResponse(new ReplyHeader(-2, lastZxid, 0), null, "response"); + cnxn.sendResponse(new ReplyHeader(ClientCnxn.PING_XID, lastZxid, 0), null, "response"); return; } case OpCode.createSession: { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java index 459f80246da..e02a63df296 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java @@ -33,6 +33,7 @@ import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,13 +52,13 @@ public class LogFormatter { public static void main(String[] args) throws Exception { if (args.length != 1) { System.err.println("USAGE: LogFormatter log_file"); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } String error = ZKUtil.validateFileInput(args[0]); if (null != error) { System.err.println(error); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } FileInputStream fis = new FileInputStream(args[0]); @@ -67,7 +68,7 @@ public static void main(String[] args) throws Exception { if (fhdr.getMagic() != FileTxnLog.TXNLOG_MAGIC) { System.err.println("Invalid magic number for " + args[0]); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } System.out.println("ZooKeeper Transactional Log File with dbid " + fhdr.getDbid() 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 d0591e57673..9cde078d360 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 @@ -34,6 +34,7 @@ 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.WatchedEvent; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.data.Id; @@ -681,7 +682,7 @@ public void sendResponse(ReplyHeader h, Record r, String tag, String cacheKey, S */ @Override public void process(WatchedEvent event) { - ReplyHeader h = new ReplyHeader(-1, -1L, 0); + ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, -1L, 0); if (LOG.isTraceEnabled()) { ZooTrace.logTraceMessage( LOG, 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 670ad83e697..442f39d173a 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 @@ -40,6 +40,7 @@ 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.WatchedEvent; import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.Stat; @@ -70,6 +71,14 @@ public class NettyServerCnxn extends ServerCnxn { public int readIssuedAfterReadComplete; + private volatile HandshakeState handshakeState = HandshakeState.NONE; + + public enum HandshakeState { + NONE, + STARTED, + FINISHED + } + NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) { super(zks); this.channel = channel; @@ -147,7 +156,7 @@ public int getSessionTimeout() { @Override public void process(WatchedEvent event) { - ReplyHeader h = new ReplyHeader(-1, -1L, 0); + ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, -1L, 0); if (LOG.isTraceEnabled()) { ZooTrace.logTraceMessage( LOG, @@ -630,4 +639,11 @@ public int getQueuedReadableBytes() { return 0; } + public void setHandshakeState(HandshakeState state) { + this.handshakeState = state; + } + + public HandshakeState getHandshakeState() { + return this.handshakeState; + } } 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 ef35837c460..78e03029fa2 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 @@ -69,6 +69,7 @@ import org.apache.zookeeper.common.SSLContextAndOptions; import org.apache.zookeeper.common.X509Exception; import org.apache.zookeeper.common.X509Exception.SSLContextException; +import org.apache.zookeeper.server.NettyServerCnxn.HandshakeState; import org.apache.zookeeper.server.auth.ProviderRegistry; import org.apache.zookeeper.server.auth.X509AuthenticationProvider; import org.apache.zookeeper.server.quorum.QuorumPeerConfig; @@ -93,6 +94,18 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory { */ private static final byte TLS_HANDSHAKE_RECORD_TYPE = 0x16; + private final AtomicInteger outstandingHandshake = new AtomicInteger(); + public static final String OUTSTANDING_HANDSHAKE_LIMIT = "zookeeper.netty.server.outstandingHandshake.limit"; + private int outstandingHandshakeLimit; + private boolean handshakeThrottlingEnabled; + + public void setOutstandingHandshakeLimit(int limit) { + outstandingHandshakeLimit = limit; + handshakeThrottlingEnabled = (secure || shouldUsePortUnification) && outstandingHandshakeLimit > 0; + LOG.info("handshakeThrottlingEnabled = {}, {} = {}", + handshakeThrottlingEnabled, OUTSTANDING_HANDSHAKE_LIMIT, outstandingHandshakeLimit); + } + private final ServerBootstrap bootstrap; private Channel parentChannel; private final ChannelGroup allChannels = new DefaultChannelGroup("zkServerCnxns", new DefaultEventExecutor()); @@ -164,6 +177,8 @@ protected SslHandler newSslHandler(ChannelHandlerContext context, SslContext ssl protected ChannelHandler newNonSslHandler(ChannelHandlerContext context) { NettyServerCnxn cnxn = Objects.requireNonNull(context.channel().attr(CONNECTION_ATTRIBUTE).get()); LOG.debug("creating plaintext handler for session {}", cnxn.getSessionId()); + // Mark handshake finished if it's a insecure cnxn + updateHandshakeCountIfStarted(cnxn); allChannels.add(context.channel()); addCnxn(cnxn); return super.newNonSslHandler(context); @@ -171,6 +186,13 @@ protected ChannelHandler newNonSslHandler(ChannelHandlerContext context) { } + private void updateHandshakeCountIfStarted(NettyServerCnxn cnxn) { + if (cnxn != null && cnxn.getHandshakeState() == HandshakeState.STARTED) { + cnxn.setHandshakeState(HandshakeState.FINISHED); + outstandingHandshake.addAndGet(-1); + } + } + /** * This is an inner class since we need to extend ChannelDuplexHandler, but * NettyServerCnxnFactory already extends ServerCnxnFactory. By making it inner @@ -202,6 +224,23 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { NettyServerCnxn cnxn = new NettyServerCnxn(channel, zkServer, NettyServerCnxnFactory.this); ctx.channel().attr(CONNECTION_ATTRIBUTE).set(cnxn); + if (handshakeThrottlingEnabled) { + // Favor to check and throttling even in dual mode which + // accepts both secure and insecure connections, since + // it's more efficient than throttling when we know it's + // a secure connection in DualModeSslHandler. + // + // From benchmark, this reduced around 15% reconnect time. + int outstandingHandshakesNum = outstandingHandshake.addAndGet(1); + if (outstandingHandshakesNum > outstandingHandshakeLimit) { + outstandingHandshake.addAndGet(-1); + channel.close(); + ServerMetrics.getMetrics().TLS_HANDSHAKE_EXCEEDED.add(1); + } else { + cnxn.setHandshakeState(HandshakeState.STARTED); + } + } + if (secure) { SslHandler sslHandler = ctx.pipeline().get(SslHandler.class); Future handshakeFuture = sslHandler.handshakeFuture(); @@ -224,6 +263,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { if (LOG.isTraceEnabled()) { LOG.trace("Channel inactive caused close {}", cnxn); } + updateHandshakeCountIfStarted(cnxn); cnxn.close(ServerCnxn.DisconnectReason.CHANNEL_DISCONNECTED); } } @@ -234,6 +274,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E NettyServerCnxn cnxn = ctx.channel().attr(CONNECTION_ATTRIBUTE).getAndSet(null); if (cnxn != null) { LOG.debug("Closing {}", cnxn); + updateHandshakeCountIfStarted(cnxn); cnxn.close(ServerCnxn.DisconnectReason.CHANNEL_CLOSED_EXCEPTION); } } @@ -339,6 +380,8 @@ final class CertificateVerifier implements GenericFutureListener * Only allow the connection to stay open if certificate passes auth */ public void operationComplete(Future future) { + updateHandshakeCountIfStarted(cnxn); + if (future.isSuccess()) { LOG.debug("Successful handshake with session 0x{}", Long.toHexString(cnxn.getSessionId())); SSLEngine eng = sslHandler.engine(); @@ -451,6 +494,8 @@ private ServerBootstrap configureBootstrapAllocator(ServerBootstrap bootstrap) { this.advancedFlowControlEnabled = Boolean.getBoolean(NETTY_ADVANCED_FLOW_CONTROL); LOG.info("{} = {}", NETTY_ADVANCED_FLOW_CONTROL, this.advancedFlowControlEnabled); + setOutstandingHandshakeLimit(Integer.getInteger(OUTSTANDING_HANDSHAKE_LIMIT, -1)); + EventLoopGroup bossGroup = NettyUtils.newNioOrEpollEventLoopGroup(NettyUtils.getClientReachableLocalInetAddressCount()); EventLoopGroup workerGroup = NettyUtils.newNioOrEpollEventLoopGroup(); ServerBootstrap bootstrap = new ServerBootstrap().group(bossGroup, workerGroup) @@ -756,4 +801,8 @@ public void setSecure(boolean secure) { public Channel getParentChannel() { return parentChannel; } + + public int getOutstandingHandshakeNum() { + return outstandingHandshake.get(); + } } 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 f7fbde301c5..0c9206647e1 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 @@ -30,6 +30,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.persistence.Util; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -230,7 +231,7 @@ private static int validateAndGetCount(String number) { private static void printUsageThenExit() { printUsage(); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); } } 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 880ac49d9b2..16a1c6f1517 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 @@ -20,6 +20,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.concurrent.LinkedBlockingQueue; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -257,7 +258,7 @@ public void shutdown() { } catch (InterruptedException e) { LOG.warn("Interrupted while waiting for {} to finish", this); //TODO apply ZOOKEEPER-575 and remove this line. - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java index 10bf444a8ee..5cd4abe3b9e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java @@ -229,6 +229,7 @@ private ServerMetrics(MetricsProvider metricsProvider) { NETTY_QUEUED_BUFFER = metricsContext.getSummary("netty_queued_buffer_capacity", DetailLevel.BASIC); DIGEST_MISMATCHES_COUNT = metricsContext.getCounter("digest_mismatches_count"); + TLS_HANDSHAKE_EXCEEDED = metricsContext.getCounter("tls_handshake_exceeded"); } /** @@ -441,6 +442,8 @@ private ServerMetrics(MetricsProvider metricsProvider) { // txns to data tree. public final Counter DIGEST_MISMATCHES_COUNT; + public final Counter TLS_HANDSHAKE_EXCEEDED; + private final MetricsProvider metricsProvider; public void resetAll() { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java index b3bf51b25bd..a276ef3d0b2 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java @@ -35,6 +35,7 @@ import org.apache.zookeeper.server.persistence.FileSnap; import org.apache.zookeeper.server.persistence.SnapStream; import org.apache.zookeeper.server.persistence.Util; +import org.apache.zookeeper.util.ServiceUtils; import org.json.simple.JSONValue; /** @@ -72,18 +73,19 @@ public static void main(String[] args) throws Exception { System.err.println("USAGE: SnapshotFormatter [-d|-json] snapshot_file"); System.err.println(" -d dump the data for each znode"); System.err.println(" -json dump znode info in json format"); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); + return; } String error = ZKUtil.validateFileInput(snapshotFile); if (null != error) { System.err.println(error); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } if (dumpData && dumpJson) { System.err.println("Cannot specify both data dump (-d) and json mode (-json) in same call"); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } new SnapshotFormatter().run(snapshotFile, dumpData, dumpJson); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java index 36a1e680be5..d1fd989b243 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java @@ -25,6 +25,7 @@ import java.text.DateFormat; import java.util.Date; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.util.ServiceUtils; public class TraceFormatter { @@ -35,7 +36,7 @@ public class TraceFormatter { public static void main(String[] args) throws IOException { if (args.length != 1) { System.err.println("USAGE: TraceFormatter trace_file"); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } FileChannel fc = new FileInputStream(args[0]).getChannel(); while (true) { 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 90ad6afeb01..65be888380e 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 @@ -80,6 +80,7 @@ import org.apache.zookeeper.server.util.RequestPathMetricsCollector; import org.apache.zookeeper.txn.CreateSessionTxn; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -511,7 +512,7 @@ public void takeSnapshot(boolean syncSnap) { LOG.error("Severe unrecoverable error, exiting", e); // This is a severe error that we cannot recover from, // so we need to exit - System.exit(ExitCode.TXNLOG_ERROR_TAKING_SNAPSHOT.getValue()); + ServiceUtils.requestSystemExit(ExitCode.TXNLOG_ERROR_TAKING_SNAPSHOT.getValue()); } long elapsed = Time.currentElapsedTime() - start; LOG.info("Snapshot taken in {} ms", elapsed); @@ -1815,6 +1816,7 @@ protected void registerMetrics() { rootContext.registerGauge("max_client_response_size", stats.getClientResponseStats()::getMaxBufferSize); rootContext.registerGauge("min_client_response_size", stats.getClientResponseStats()::getMinBufferSize); + rootContext.registerGauge("outstanding_tls_handshake", this::getOutstandingHandshakeNum); } protected void unregisterMetrics() { @@ -2074,4 +2076,11 @@ private boolean buffer2Record(ByteBuffer request, Record record) { return rv; } + public int getOutstandingHandshakeNum() { + if (serverCnxnFactory instanceof NettyServerCnxnFactory) { + return ((NettyServerCnxnFactory) serverCnxnFactory).getOutstandingHandshakeNum(); + } else { + return 0; + } + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java index 4a1b6009596..7ddb1deb9e6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java @@ -35,6 +35,7 @@ import org.apache.zookeeper.server.persistence.FileTxnSnapLog.DatadirException; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; import org.apache.zookeeper.server.util.JvmPauseMonitor; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,29 +70,29 @@ public static void main(String[] args) { LOG.info(USAGE); System.err.println(USAGE); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } catch (ConfigException e) { LOG.error("Invalid config, exiting abnormally", e); System.err.println("Invalid config, exiting abnormally"); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } catch (DatadirException e) { LOG.error("Unable to access datadir, exiting abnormally", e); System.err.println("Unable to access datadir, exiting abnormally"); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.UNABLE_TO_ACCESS_DATADIR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNABLE_TO_ACCESS_DATADIR.getValue()); } catch (AdminServerException e) { LOG.error("Unable to start AdminServer, exiting abnormally", e); System.err.println("Unable to start AdminServer, exiting abnormally"); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.ERROR_STARTING_ADMIN_SERVER.getValue()); + ServiceUtils.requestSystemExit(ExitCode.ERROR_STARTING_ADMIN_SERVER.getValue()); } catch (Exception e) { LOG.error("Unexpected exception, exiting abnormally", e); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); } LOG.info("Exiting normally"); - System.exit(ExitCode.EXECUTION_FINISHED.getValue()); + ServiceUtils.requestSystemExit(ExitCode.EXECUTION_FINISHED.getValue()); } protected void initializeAndRun(String[] args) throws ConfigException, IOException, AdminServerException { @@ -169,7 +170,9 @@ public void runFromConfig(ServerConfig config) throws IOException, AdminServerEx zkServer.getZKDatabase(), zkServer.firstProcessor, Integer.getInteger("znode.container.checkIntervalMs", (int) TimeUnit.MINUTES.toMillis(1)), - Integer.getInteger("znode.container.maxPerMinute", 10000)); + Integer.getInteger("znode.container.maxPerMinute", 10000), + Long.getLong("znode.container.maxNeverUsedIntervalMs", 0) + ); containerManager.start(); ZKAuditProvider.addZKStartStopAuditLog(); 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 3fe4ea5643c..dbcdabf849b 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 @@ -51,6 +51,7 @@ import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer; import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; +import org.apache.zookeeper.server.util.ZxidUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -809,10 +810,21 @@ public CommandResponse run(ZooKeeperServer zkServer, Map kwargs) QuorumVerifier qv = peer.getQuorumVerifier(); QuorumPeer.QuorumServer voter = qv.getVotingMembers().get(peer.getId()); - boolean voting = (voter != null - && voter.addr.equals(peer.getQuorumAddress()) - && voter.electionAddr.equals(peer.getElectionAddress())); + boolean voting = ( + voter != null + && voter.addr.equals(peer.getQuorumAddress()) + && voter.electionAddr.equals(peer.getElectionAddress()) + ); + response.put("myid", zkServer.getConf().getServerId()); + response.put("is_leader", zkServer instanceof LeaderZooKeeperServer); + response.put("quorum_address", peer.getQuorumAddress()); + response.put("election_address", peer.getElectionAddress()); + response.put("client_address", peer.getClientAddress()); response.put("voting", voting); + long lastProcessedZxid = zkServer.getZKDatabase().getDataTreeLastProcessedZxid(); + response.put("last_zxid", "0x" + ZxidUtils.zxidToString(lastProcessedZxid)); + response.put("zab_epoch", ZxidUtils.getEpochFromZxid(lastProcessedZxid)); + response.put("zab_counter", ZxidUtils.getCounterFromZxid(lastProcessedZxid)); response.put("zabstate", zabState.name().toLowerCase()); } else { response.put("voting", false); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java index bb95d006ac9..984fb03e281 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java @@ -58,6 +58,7 @@ import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.Txn; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; public class TxnLogToolkit implements Closeable { @@ -126,7 +127,7 @@ public static void main(String[] args) throws Exception { printHelpAndExit(e.getExitCode(), e.getOptions()); } catch (TxnLogToolkitException e) { System.err.println(e.getMessage()); - System.exit(e.getExitCode()); + ServiceUtils.requestSystemExit(e.getExitCode()); } } @@ -424,7 +425,7 @@ private static TxnLogToolkit parseCommandLine(String[] args) throws TxnLogToolki private static void printHelpAndExit(int exitCode, Options options) { HelpFormatter help = new HelpFormatter(); help.printHelp(120, "TxnLogToolkit [-dhrvc] (-z )", "", options, ""); - System.exit(exitCode); + ServiceUtils.requestSystemExit(exitCode); } private void printStat() { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java index 4b786dab0ea..12e552f9d7d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java @@ -34,6 +34,7 @@ import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,7 +101,7 @@ public void commit(long zxid) { if (firstElementZxid != zxid) { LOG.error("Committing zxid 0x" + Long.toHexString(zxid) + " but next pending txn 0x" + Long.toHexString(firstElementZxid)); - System.exit(ExitCode.UNMATCHED_TXN_COMMIT.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNMATCHED_TXN_COMMIT.getValue()); } Request request = pendingTxns.remove(); request.logLatency(ServerMetrics.getMetrics().COMMIT_PROPAGATION_LATENCY); 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 a0e32f7f51a..27de4519177 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 @@ -81,7 +81,9 @@ private synchronized void setupContainerManager() { getZKDatabase(), prepRequestProcessor, Integer.getInteger("znode.container.checkIntervalMs", (int) TimeUnit.MINUTES.toMillis(1)), - Integer.getInteger("znode.container.maxPerMinute", 10000)); + Integer.getInteger("znode.container.maxPerMinute", 10000), + Long.getLong("znode.container.maxNeverUsedIntervalMs", 0) + ); } @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 1efe1cbb9c1..3d6ff1b79c1 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 @@ -58,6 +58,7 @@ import org.apache.zookeeper.server.util.ZxidUtils; import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -530,14 +531,13 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception { if (!truncated) { // not able to truncate the log LOG.error("Not able to truncate the log 0x{}", Long.toHexString(qp.getZxid())); - System.exit(ExitCode.QUORUM_PACKET_ERROR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.QUORUM_PACKET_ERROR.getValue()); } zk.getZKDatabase().setlastProcessedZxid(qp.getZxid()); } else { LOG.error("Got unexpected packet from leader: {}, exiting ... ", LearnerHandler.packetToString(qp)); - System.exit(ExitCode.QUORUM_PACKET_ERROR.getValue()); - + ServiceUtils.requestSystemExit(ExitCode.QUORUM_PACKET_ERROR.getValue()); } zk.getZKDatabase().initConfigInZKDatabase(self.getQuorumVerifier()); zk.createSessionTracker(); 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 ff248415e4a..55c1dc27139 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 @@ -66,6 +66,7 @@ import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.ConfigUtils; import org.apache.zookeeper.util.CircularBlockingQueue; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -862,7 +863,7 @@ public class Listener extends ZooKeeperThread { private static final int DEFAULT_PORT_BIND_MAX_RETRY = 3; private final int portBindMaxRetry; - private Runnable socketBindErrorHandler = () -> System.exit(ExitCode.UNABLE_TO_BIND_QUORUM_PORT.getValue()); + private Runnable socketBindErrorHandler = () -> ServiceUtils.requestSystemExit(ExitCode.UNABLE_TO_BIND_QUORUM_PORT.getValue()); private List listenerHandlers; private final AtomicBoolean socketException; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index bf97488e97b..054a47dcd69 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -38,6 +38,7 @@ import org.apache.zookeeper.server.persistence.FileTxnSnapLog.DatadirException; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; import org.apache.zookeeper.server.util.JvmPauseMonitor; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,29 +93,29 @@ public static void main(String[] args) { LOG.info(USAGE); System.err.println(USAGE); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } catch (ConfigException e) { LOG.error("Invalid config, exiting abnormally", e); System.err.println("Invalid config, exiting abnormally"); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.INVALID_INVOCATION.getValue()); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); } catch (DatadirException e) { LOG.error("Unable to access datadir, exiting abnormally", e); System.err.println("Unable to access datadir, exiting abnormally"); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.UNABLE_TO_ACCESS_DATADIR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNABLE_TO_ACCESS_DATADIR.getValue()); } catch (AdminServerException e) { LOG.error("Unable to start AdminServer, exiting abnormally", e); System.err.println("Unable to start AdminServer, exiting abnormally"); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.ERROR_STARTING_ADMIN_SERVER.getValue()); + ServiceUtils.requestSystemExit(ExitCode.ERROR_STARTING_ADMIN_SERVER.getValue()); } catch (Exception e) { LOG.error("Unexpected exception, exiting abnormally", e); ZKAuditProvider.addServerStartFailureAuditLog(); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); } LOG.info("Exiting normally"); - System.exit(ExitCode.EXECUTION_FINISHED.getValue()); + ServiceUtils.requestSystemExit(ExitCode.EXECUTION_FINISHED.getValue()); } protected void initializeAndRun(String[] args) throws ConfigException, IOException, AdminServerException { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java index 3a070788ac2..e0e52a9d88d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java @@ -36,6 +36,7 @@ import org.apache.zookeeper.server.persistence.FileHeader; import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; /** * this class will chop the log at the specified zxid @@ -49,7 +50,7 @@ public static void main(String[] args) { System.out.println("Usage: LogChopper zxid_to_chop_to txn_log_to_chop chopped_filename"); System.out.println(" this program will read the txn_log_to_chop file and copy all the transactions"); System.out.println(" from it up to (and including) the given zxid into chopped_filename."); - System.exit(rc.getValue()); + ServiceUtils.requestSystemExit(rc.getValue()); } String txnLog = args[1]; String choppedLog = args[2]; @@ -63,7 +64,7 @@ public static void main(String[] args) { } catch (Exception e) { System.out.println("Got exception: " + e.getMessage()); } - System.exit(rc.getValue()); + ServiceUtils.requestSystemExit(rc.getValue()); } public static boolean chop(InputStream is, OutputStream os, long zxid) throws IOException { 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 new file mode 100644 index 00000000000..68a25ebb487 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/util/ServiceUtils.java @@ -0,0 +1,77 @@ +/* + * 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.util; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.Objects; +import java.util.function.Consumer; +import org.apache.zookeeper.server.ExitCode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utilities for service management. + */ +public abstract class ServiceUtils { + + private static final Logger LOG = LoggerFactory.getLogger(ServiceUtils.class); + + private ServiceUtils() { + } + + /** + * Default strategy for shutting down the JVM. + */ + @SuppressFBWarnings("DM_EXIT") + public static final Consumer SYSTEM_EXIT = (code) -> { + LOG.error("Exiting JVM with code {}", code); + System.exit(code); + }; + + /** + * No-op strategy, useful for tests. + */ + public static final Consumer LOG_ONLY = (code) -> { + LOG.error("Fatal error, JVM should exit with code {}. " + + "Actually System.exit is disabled", code); + }; + + private static Consumer systemExitProcedure = SYSTEM_EXIT; + + /** + * Override system callback. Useful for preventing the JVM to exit in tests + * or in applications that are running an in-process ZooKeeper server. + * + * @param systemExitProcedure + */ + public static void setSystemExitProcedure(Consumer systemExitProcedure) { + Objects.requireNonNull(systemExitProcedure); + ServiceUtils.systemExitProcedure = systemExitProcedure; + } + + /** + * Force shutdown of the JVM using System.exit. + * + * @param code the exit code + * @see ExitCode + */ + public static void requestSystemExit(int code) { + systemExitProcedure.accept(code); + } + +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java b/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java index c7f9fd96236..3e9e62aee1c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java @@ -18,6 +18,7 @@ package org.apache.zookeeper.version.util; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.File; import java.io.FileWriter; import java.io.IOException; @@ -25,6 +26,7 @@ import java.util.regex.Pattern; import org.apache.zookeeper.server.ExitCode; +@SuppressFBWarnings("DM_EXIT") public class VerGen { private static final String PACKAGE_NAME = "org.apache.zookeeper.version"; @@ -123,7 +125,7 @@ public static void generateFile(File outputDir, Version version, String rev, Str w.write("}\n"); } catch (IOException e) { System.out.println("Unable to generate version.VersionInfoMain file: " + e.getMessage()); - System.exit(1); + System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); } } 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 d9d481aa24d..78e3cd79e82 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java @@ -22,6 +22,7 @@ import static org.junit.Assert.fail; import java.io.File; import java.time.LocalDateTime; +import org.apache.zookeeper.util.ServiceUtils; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.rules.TestWatcher; @@ -42,6 +43,11 @@ public class ZKTestCase { protected static final File testBaseDir = new File(System.getProperty("build.test.dir", "build")); private static final Logger LOG = LoggerFactory.getLogger(ZKTestCase.class); + static { + // Disable System.exit in tests. + ServiceUtils.setSystemExitProcedure(ServiceUtils.LOG_ONLY); + } + private String testName; protected String getTestName() { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateContainerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateContainerTest.java index 83c7f0b3db9..03f9bcce210 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateContainerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/CreateContainerTest.java @@ -31,7 +31,10 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -45,11 +48,24 @@ public class CreateContainerTest extends ClientBase { private ZooKeeper zk; + private Semaphore completedContainerDeletions; @Override public void setUp() throws Exception { super.setUp(); zk = createClient(); + + completedContainerDeletions = new Semaphore(0); + ZKDatabase testDatabase = new ZKDatabase(serverFactory.zkServer.getZKDatabase().snapLog) { + @Override + public void addCommittedProposal(Request request) { + super.addCommittedProposal(request); + if (request.type == ZooDefs.OpCode.deleteContainer) { + completedContainerDeletions.release(); + } + } + }; + serverFactory.zkServer.setZKDatabase(testDatabase); } @Override @@ -95,8 +111,7 @@ public void testSimpleDeletion() throws KeeperException, InterruptedException { ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer().getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); containerManager.checkContainers(); - Thread.sleep(1000); - + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); assertNull("Container should have been deleted", zk.exists("/foo", false)); } @@ -123,8 +138,7 @@ public void testMultiWithContainer() throws KeeperException, InterruptedExceptio ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer().getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); containerManager.checkContainers(); - Thread.sleep(1000); - + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); assertNull("Container should have been deleted", zk.exists("/foo", false)); createContainer = Op.create("/foo", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); @@ -134,8 +148,7 @@ public void testMultiWithContainer() throws KeeperException, InterruptedExceptio containerManager.checkContainers(); - Thread.sleep(1000); - + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); assertNull("Container should have been deleted", zk.exists("/foo", false)); } @@ -157,8 +170,7 @@ public void processResult(int rc, String path, Object ctx, String name, Stat sta ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer().getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); containerManager.checkContainers(); - Thread.sleep(1000); - + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); assertNull("Container should have been deleted", zk.exists("/foo", false)); } @@ -171,9 +183,9 @@ public void testCascadingDeletion() throws KeeperException, InterruptedException ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer().getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); containerManager.checkContainers(); - Thread.sleep(1000); + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); containerManager.checkContainers(); - Thread.sleep(1000); + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); assertNull("Container should have been deleted", zk.exists("/foo/bar", false)); assertNull("Container should have been deleted", zk.exists("/foo", false)); @@ -191,8 +203,8 @@ protected Collection getCandidates() { } }; containerManager.checkContainers(); - Thread.sleep(1000); + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); assertNotNull("Container should have not been deleted", zk.exists("/foo", false)); } @@ -237,6 +249,54 @@ public Void call() throws Exception { assertEquals(queue.poll(5, TimeUnit.SECONDS), "/four"); } + @Test(timeout = 30000) + public void testMaxNeverUsedInterval() throws KeeperException, InterruptedException { + zk.create("/foo", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + AtomicLong elapsed = new AtomicLong(0); + AtomicInteger deletesQty = new AtomicInteger(0); + ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer().getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100, 1000) { + @Override + protected void postDeleteRequest(Request request) throws RequestProcessor.RequestProcessorException { + deletesQty.incrementAndGet(); + super.postDeleteRequest(request); + } + + @Override + protected long getElapsed(DataNode node) { + return elapsed.get(); + } + }; + containerManager.checkContainers(); // elapsed time will appear to be 0 - container will not get deleted + assertEquals(deletesQty.get(), 0); + assertNotNull("Container should not have been deleted", zk.exists("/foo", false)); + + elapsed.set(10000); + containerManager.checkContainers(); // elapsed time will appear to be 10000 - container should get deleted + assertTrue(completedContainerDeletions.tryAcquire(1, TimeUnit.SECONDS)); + assertNull("Container should have been deleted", zk.exists("/foo", false)); + } + + @Test(timeout = 30000) + public void testZeroMaxNeverUsedInterval() throws KeeperException, InterruptedException { + zk.create("/foo", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + AtomicInteger deletesQty = new AtomicInteger(0); + ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer().getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100, 0) { + @Override + protected void postDeleteRequest(Request request) throws RequestProcessor.RequestProcessorException { + deletesQty.incrementAndGet(); + super.postDeleteRequest(request); + } + + @Override + protected long getElapsed(DataNode node) { + return 10000; // some number greater than 0 + } + }; + containerManager.checkContainers(); // elapsed time will appear to be 0 - container will not get deleted + assertEquals(deletesQty.get(), 0); + assertNotNull("Container should not have been deleted", zk.exists("/foo", false)); + } + private void createNoStatVerifyResult(String newName) throws KeeperException, InterruptedException { assertNull("Node existed before created", zk.exists(newName, false)); zk.create(newName, newName.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java index 144ca3ba2c6..afb97b14ab3 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java @@ -19,11 +19,49 @@ package org.apache.zookeeper.server; import java.net.InetSocketAddress; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.ClientX509Util; +import org.apache.zookeeper.server.metric.SimpleCounter; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.SSLAuthTest; +import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class NettyServerCnxnFactoryTest { + +public class NettyServerCnxnFactoryTest extends ClientBase { + + private static final Logger LOG = LoggerFactory + .getLogger(NettyServerCnxnFactoryTest.class); + + final LinkedBlockingQueue zks = new LinkedBlockingQueue(); + + @Override + public void setUp() throws Exception { + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, + "org.apache.zookeeper.server.NettyServerCnxnFactory"); + super.setUp(); + } + + @Override + public void tearDown() throws Exception { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + + // clean up + for (ZooKeeper zk : zks) { + zk.close(); + } + super.tearDown(); + } @Test public void testRebind() throws Exception { @@ -58,4 +96,63 @@ public void testRebindIPv4IPv6() throws Exception { Assert.assertTrue(factory.getParentChannel().isActive()); } + @Test + public void testOutstandingHandshakeLimit() throws Exception { + + SimpleCounter tlsHandshakeExceeded = (SimpleCounter) ServerMetrics.getMetrics().TLS_HANDSHAKE_EXCEEDED; + tlsHandshakeExceeded.reset(); + Assert.assertEquals(tlsHandshakeExceeded.get(), 0); + + ClientX509Util x509Util = SSLAuthTest.setUpSecure(); + NettyServerCnxnFactory factory = (NettyServerCnxnFactory) serverFactory; + factory.setSecure(true); + factory.setOutstandingHandshakeLimit(10); + + int threadNum = 3; + int cnxnPerThread = 10; + Thread[] cnxnWorker = new Thread[threadNum]; + + AtomicInteger cnxnCreated = new AtomicInteger(0); + CountDownLatch latch = new CountDownLatch(1); + + for (int i = 0; i < cnxnWorker.length; i++) { + cnxnWorker[i] = new Thread() { + @Override + public void run() { + for (int i = 0; i < cnxnPerThread; i++) { + try { + zks.add(new ZooKeeper(hostPort, 3000, new Watcher() { + @Override + public void process(WatchedEvent event) { + int created = cnxnCreated.addAndGet(1); + if (created == threadNum * cnxnPerThread) { + latch.countDown(); + } + } + })); + } catch (Exception e) { + LOG.info("Error while creating zk client", e); + } + } + } + }; + cnxnWorker[i].start(); + } + + Assert.assertThat(latch.await(3, TimeUnit.SECONDS), Matchers.is(true)); + LOG.info("created {} connections", threadNum * cnxnPerThread); + + // Assert throttling not 0 + long handshakeThrottledNum = tlsHandshakeExceeded.get(); + LOG.info("TLS_HANDSHAKE_EXCEEDED: {}", handshakeThrottledNum); + Assert.assertThat("The number of handshake throttled should be " + + "greater than 0", handshakeThrottledNum, Matchers.greaterThan(0L)); + + // Assert there is no outstanding handshake anymore + int outstandingHandshakeNum = factory.getOutstandingHandshakeNum(); + LOG.info("outstanding handshake is {}", outstandingHandshakeNum); + Assert.assertThat("The outstanding handshake number should be 0 " + + "after all cnxns established", outstandingHandshakeNum, Matchers.is(0)); + + } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java index d4f85e84ca1..aa7d113fead 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java @@ -155,7 +155,31 @@ public void testLastSnapshot() throws IOException, InterruptedException { @Test public void testMonitor() throws IOException, InterruptedException { - ArrayList fields = new ArrayList<>(Arrays.asList(new Field("version", String.class), new Field("avg_latency", Double.class), new Field("max_latency", Long.class), new Field("min_latency", Long.class), new Field("packets_received", Long.class), new Field("packets_sent", Long.class), new Field("num_alive_connections", Integer.class), new Field("outstanding_requests", Long.class), new Field("server_state", String.class), new Field("znode_count", Integer.class), new Field("watch_count", Integer.class), new Field("ephemerals_count", Integer.class), new Field("approximate_data_size", Long.class), new Field("open_file_descriptor_count", Long.class), new Field("max_file_descriptor_count", Long.class), new Field("last_client_response_size", Integer.class), new Field("max_client_response_size", Integer.class), new Field("min_client_response_size", Integer.class), new Field("uptime", Long.class), new Field("global_sessions", Long.class), new Field("local_sessions", Long.class), new Field("connection_drop_probability", Double.class))); + ArrayList fields = new ArrayList<>(Arrays.asList( + new Field("version", String.class), + new Field("avg_latency", Double.class), + new Field("max_latency", Long.class), + new Field("min_latency", Long.class), + new Field("packets_received", Long.class), + new Field("packets_sent", Long.class), + new Field("num_alive_connections", Integer.class), + new Field("outstanding_requests", Long.class), + new Field("server_state", String.class), + new Field("znode_count", Integer.class), + new Field("watch_count", Integer.class), + new Field("ephemerals_count", Integer.class), + new Field("approximate_data_size", Long.class), + new Field("open_file_descriptor_count", Long.class), + new Field("max_file_descriptor_count", Long.class), + new Field("last_client_response_size", Integer.class), + new Field("max_client_response_size", Integer.class), + new Field("min_client_response_size", Integer.class), + new Field("uptime", Long.class), + new Field("global_sessions", Long.class), + new Field("local_sessions", Long.class), + new Field("connection_drop_probability", Double.class), + new Field("outstanding_tls_handshake", Integer.class) + )); Map metrics = MetricsUtils.currentServerMetrics(); for (String metric : metrics.keySet()) { 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 cdc3b830470..2455738e4b8 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 @@ -43,6 +43,7 @@ import java.util.Random; import org.apache.jute.InputArchive; import org.apache.jute.OutputArchive; +import org.apache.zookeeper.ClientCnxn; import org.apache.zookeeper.MockPacket; import org.apache.zookeeper.ZKParameterized; import org.apache.zookeeper.ZooDefs; @@ -251,7 +252,7 @@ private ByteBuffer createWatchesMessage() { SetWatches sw = new SetWatches(1L, dataWatches, existWatches, childWatches); RequestHeader h = new RequestHeader(); h.setType(ZooDefs.OpCode.setWatches); - h.setXid(-8); + h.setXid(ClientCnxn.SET_WATCHES_XID); MockPacket p = new MockPacket(h, new ReplyHeader(), sw, null, null); return p.createAndReturnBB(); }