http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java
deleted file mode 100644
index 8550626..0000000
--- a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java
+++ /dev/null
@@ -1,477 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.client;
-
-import java.io.IOException;
-import java.security.PrivilegedActionException;
-import java.security.PrivilegedExceptionAction;
-
-import javax.security.auth.Subject;
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.Configuration;
-import javax.security.auth.login.LoginException;
-import javax.security.sasl.SaslClient;
-import javax.security.sasl.SaslException;
-
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.ClientCnxn;
-import org.apache.zookeeper.Login;
-import org.apache.zookeeper.SaslClientCallbackHandler;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.GetSASLRequest;
-import org.apache.zookeeper.proto.SetSASLResponse;
-
-import org.apache.zookeeper.util.SecurityUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class manages SASL authentication for the client. It
- * allows ClientCnxn to authenticate using SASL with a ZooKeeper server.
- */
-public class ZooKeeperSaslClient {
- /**
- * @deprecated Use {@link ZKClientConfig#LOGIN_CONTEXT_NAME_KEY}
- * instead.
- */
- @Deprecated
- public static final String LOGIN_CONTEXT_NAME_KEY = "zookeeper.sasl.clientconfig";
- /**
- * @deprecated Use {@link ZKClientConfig#ENABLE_CLIENT_SASL_KEY}
- * instead.
- */
- @Deprecated
- public static final String ENABLE_CLIENT_SASL_KEY = "zookeeper.sasl.client";
- /**
- * @deprecated Use {@link ZKClientConfig#ENABLE_CLIENT_SASL_DEFAULT}
- * instead.
- */
- @Deprecated
- public static final String ENABLE_CLIENT_SASL_DEFAULT = "true";
- private volatile boolean initializedLogin = false;
-
- /**
- * Returns true if the SASL client is enabled. By default, the client
- * is enabled but can be disabled by setting the system property
- * <code>zookeeper.sasl.client</code> to <code>false</code>. See
- * ZOOKEEPER-1657 for more information.
- *
- * @return true if the SASL client is enabled.
- * @deprecated Use {@link ZKClientConfig#isSaslClientEnabled} instead
- */
- @Deprecated
- public static boolean isEnabled() {
- return Boolean.valueOf(System.getProperty(
- ZKClientConfig.ENABLE_CLIENT_SASL_KEY,
- ZKClientConfig.ENABLE_CLIENT_SASL_DEFAULT));
- }
-
- private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperSaslClient.class);
- private Login login = null;
- private SaslClient saslClient;
- private boolean isSASLConfigured = true;
- private final ZKClientConfig clientConfig;
-
- private byte[] saslToken = new byte[0];
-
- public enum SaslState {
- INITIAL,INTERMEDIATE,COMPLETE,FAILED
- }
-
- private SaslState saslState = SaslState.INITIAL;
-
- private boolean gotLastPacket = false;
- /** informational message indicating the current configuration status */
- private final String configStatus;
-
- public SaslState getSaslState() {
- return saslState;
- }
-
- public String getLoginContext() {
- if (login != null)
- return login.getLoginContextName();
- return null;
- }
-
- public ZooKeeperSaslClient(final String serverPrincipal, ZKClientConfig clientConfig) throws LoginException {
- /**
- * ZOOKEEPER-1373: allow system property to specify the JAAS
- * configuration section that the zookeeper client should use.
- * Default to "Client".
- */
- String clientSection = clientConfig.getProperty(
- ZKClientConfig.LOGIN_CONTEXT_NAME_KEY,
- ZKClientConfig.LOGIN_CONTEXT_NAME_KEY_DEFAULT);
- this.clientConfig = clientConfig;
- // Note that 'Configuration' here refers to javax.security.auth.login.Configuration.
- AppConfigurationEntry entries[] = null;
- RuntimeException runtimeException = null;
- try {
- entries = Configuration.getConfiguration()
- .getAppConfigurationEntry(clientSection);
- } catch (SecurityException e) {
- // handle below: might be harmless if the user doesn't intend to use JAAS authentication.
- runtimeException = e;
- } catch (IllegalArgumentException e) {
- // third party customized getAppConfigurationEntry could throw IllegalArgumentException when JAAS
- // configuration isn't set. We can reevaluate whether to catch RuntimeException instead when more
- // different types of RuntimeException found
- runtimeException = e;
- }
- if (entries != null) {
- this.configStatus = "Will attempt to SASL-authenticate using Login Context section '" + clientSection + "'";
- this.saslClient = createSaslClient(serverPrincipal, clientSection);
- } else {
- // Handle situation of clientSection's being null: it might simply because the client does not intend to
- // use SASL, so not necessarily an error.
- saslState = SaslState.FAILED;
- String explicitClientSection = clientConfig
- .getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY);
- if (explicitClientSection != null) {
- // If the user explicitly overrides the default Login Context, they probably expected SASL to
- // succeed. But if we got here, SASL failed.
- if (runtimeException != null) {
- throw new LoginException(
- "Zookeeper client cannot authenticate using the "
- + explicitClientSection
- + " section of the supplied JAAS configuration: '"
- + clientConfig.getJaasConfKey() + "' because of a "
- + "RuntimeException: " + runtimeException);
- } else {
- throw new LoginException("Client cannot SASL-authenticate because the specified JAAS configuration " +
- "section '" + explicitClientSection + "' could not be found.");
- }
- } else {
- // The user did not override the default context. It might be that they just don't intend to use SASL,
- // so log at INFO, not WARN, since they don't expect any SASL-related information.
- String msg = "Will not attempt to authenticate using SASL ";
- if (runtimeException != null) {
- msg += "(" + runtimeException + ")";
- } else {
- msg += "(unknown error)";
- }
- this.configStatus = msg;
- this.isSASLConfigured = false;
- }
- if (clientConfig.getJaasConfKey() != null) {
- // Again, the user explicitly set something SASL-related, so
- // they probably expected SASL to succeed.
- if (runtimeException != null) {
- throw new LoginException(
- "Zookeeper client cannot authenticate using the '"
- + clientConfig.getProperty(
- ZKClientConfig.LOGIN_CONTEXT_NAME_KEY,
- ZKClientConfig.LOGIN_CONTEXT_NAME_KEY_DEFAULT)
- + "' section of the supplied JAAS configuration: '"
- + clientConfig.getJaasConfKey() + "' because of a "
- + "RuntimeException: " + runtimeException);
- } else {
- throw new LoginException(
- "No JAAS configuration section named '"
- + clientConfig.getProperty(
- ZKClientConfig.LOGIN_CONTEXT_NAME_KEY,
- ZKClientConfig.LOGIN_CONTEXT_NAME_KEY_DEFAULT)
- + "' was found in specified JAAS configuration file: '"
- + clientConfig.getJaasConfKey() + "'.");
- }
- }
- }
- }
-
- /**
- * @return informational message indicating the current configuration status.
- */
- public String getConfigStatus() {
- return configStatus;
- }
-
- public boolean isComplete() {
- return (saslState == SaslState.COMPLETE);
- }
-
- public boolean isFailed() {
- return (saslState == SaslState.FAILED);
- }
-
- public static class ServerSaslResponseCallback implements AsyncCallback.DataCallback {
- public void processResult(int rc, String path, Object ctx, byte data[], Stat stat) {
- // processResult() is used by ClientCnxn's sendThread to respond to
- // data[] contains the Zookeeper Server's SASL token.
- // ctx is the ZooKeeperSaslClient object. We use this object's respondToServer() method
- // to reply to the Zookeeper Server's SASL token
- ZooKeeperSaslClient client = ((ClientCnxn)ctx).zooKeeperSaslClient;
- if (client == null) {
- LOG.warn("sasl client was unexpectedly null: cannot respond to Zookeeper server.");
- return;
- }
- byte[] usedata = data;
- if (data != null) {
- LOG.debug("ServerSaslResponseCallback(): saslToken server response: (length="+usedata.length+")");
- }
- else {
- usedata = new byte[0];
- LOG.debug("ServerSaslResponseCallback(): using empty data[] as server response (length="+usedata.length+")");
- }
- client.respondToServer(usedata, (ClientCnxn)ctx);
- }
- }
-
- private SaslClient createSaslClient(final String servicePrincipal, final String loginContext)
- throws LoginException {
- try {
- if (!initializedLogin) {
- synchronized (this) {
- if (login == null) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("JAAS loginContext is: " + loginContext);
- }
- // note that the login object is static: it's shared amongst all zookeeper-related connections.
- // in order to ensure the login is initialized only once, it must be synchronized the code snippet.
- login = new Login(loginContext, new SaslClientCallbackHandler(null, "Client"), clientConfig);
- login.startThreadIfNeeded();
- initializedLogin = true;
- }
- }
- }
- return SecurityUtils.createSaslClient(login.getSubject(),
- servicePrincipal, "zookeeper", "zk-sasl-md5", LOG, "Client");
- } catch (LoginException e) {
- // We throw LoginExceptions...
- throw e;
- } catch (Exception e) {
- // ..but consume (with a log message) all other types of exceptions.
- LOG.error("Exception while trying to create SASL client: " + e);
- return null;
- }
- }
-
- public void respondToServer(byte[] serverToken, ClientCnxn cnxn) {
- if (saslClient == null) {
- LOG.error("saslClient is unexpectedly null. Cannot respond to server's SASL message; ignoring.");
- return;
- }
-
- if (!(saslClient.isComplete())) {
- try {
- saslToken = createSaslToken(serverToken);
- if (saslToken != null) {
- sendSaslPacket(saslToken, cnxn);
- }
- } catch (SaslException e) {
- LOG.error("SASL authentication failed using login context '" +
- this.getLoginContext() + "' with exception: {}", e);
- saslState = SaslState.FAILED;
- gotLastPacket = true;
- }
- }
-
- if (saslClient.isComplete()) {
- // GSSAPI: server sends a final packet after authentication succeeds
- // or fails.
- if ((serverToken == null) && (saslClient.getMechanismName().equals("GSSAPI")))
- gotLastPacket = true;
- // non-GSSAPI: no final packet from server.
- if (!saslClient.getMechanismName().equals("GSSAPI")) {
- gotLastPacket = true;
- }
- // SASL authentication is completed, successfully or not:
- // enable the socket's writable flag so that any packets waiting for authentication to complete in
- // the outgoing queue will be sent to the Zookeeper server.
- cnxn.saslCompleted();
- }
- }
-
- private byte[] createSaslToken() throws SaslException {
- saslState = SaslState.INTERMEDIATE;
- return createSaslToken(saslToken);
- }
-
- private byte[] createSaslToken(final byte[] saslToken) throws SaslException {
- if (saslToken == null) {
- // TODO: introspect about runtime environment (such as jaas.conf)
- saslState = SaslState.FAILED;
- throw new SaslException("Error in authenticating with a Zookeeper Quorum member: the quorum member's saslToken is null.");
- }
-
- Subject subject = login.getSubject();
- if (subject != null) {
- synchronized(login) {
- try {
- final byte[] retval =
- Subject.doAs(subject, new PrivilegedExceptionAction<byte[]>() {
- public byte[] run() throws SaslException {
- LOG.debug("saslClient.evaluateChallenge(len="+saslToken.length+")");
- return saslClient.evaluateChallenge(saslToken);
- }
- });
- return retval;
- }
- catch (PrivilegedActionException e) {
- String error = "An error: (" + e + ") occurred when evaluating Zookeeper Quorum Member's " +
- " received SASL token.";
- // Try to provide hints to use about what went wrong so they can fix their configuration.
- // TODO: introspect about e: look for GSS information.
- final String UNKNOWN_SERVER_ERROR_TEXT =
- "(Mechanism level: Server not found in Kerberos database (7) - UNKNOWN_SERVER)";
- if (e.toString().indexOf(UNKNOWN_SERVER_ERROR_TEXT) > -1) {
- error += " This may be caused by Java's being unable to resolve the Zookeeper Quorum Member's" +
- " hostname correctly. You may want to try to adding" +
- " '-Dsun.net.spi.nameservice.provider.1=dns,sun' to your client's JVMFLAGS environment.";
- }
- error += " Zookeeper Client will go to AUTH_FAILED state.";
- LOG.error(error);
- saslState = SaslState.FAILED;
- throw new SaslException(error);
- }
- }
- }
- else {
- throw new SaslException("Cannot make SASL token without subject defined. " +
- "For diagnosis, please look for WARNs and ERRORs in your log related to the Login class.");
- }
- }
-
- private void sendSaslPacket(byte[] saslToken, ClientCnxn cnxn)
- throws SaslException{
- if (LOG.isDebugEnabled()) {
- LOG.debug("ClientCnxn:sendSaslPacket:length="+saslToken.length);
- }
-
- GetSASLRequest request = new GetSASLRequest();
- request.setToken(saslToken);
- SetSASLResponse response = new SetSASLResponse();
- ServerSaslResponseCallback cb = new ServerSaslResponseCallback();
-
- try {
- cnxn.sendPacket(request,response,cb, ZooDefs.OpCode.sasl);
- } catch (IOException e) {
- throw new SaslException("Failed to send SASL packet to server.",
- e);
- }
- }
-
- private void sendSaslPacket(ClientCnxn cnxn) throws SaslException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("ClientCnxn:sendSaslPacket:length="+saslToken.length);
- }
- GetSASLRequest request = new GetSASLRequest();
- request.setToken(createSaslToken());
- SetSASLResponse response = new SetSASLResponse();
- ServerSaslResponseCallback cb = new ServerSaslResponseCallback();
- try {
- cnxn.sendPacket(request,response,cb, ZooDefs.OpCode.sasl);
- } catch (IOException e) {
- throw new SaslException("Failed to send SASL packet to server due " +
- "to IOException:", e);
- }
- }
-
- // used by ClientCnxn to know whether to emit a SASL-related event: either AuthFailed or SaslAuthenticated,
- // or none, if not ready yet. Sets saslState to COMPLETE as a side-effect.
- public KeeperState getKeeperState() {
- if (saslClient != null) {
- if (saslState == SaslState.FAILED) {
- return KeeperState.AuthFailed;
- }
- if (saslClient.isComplete()) {
- if (saslState == SaslState.INTERMEDIATE) {
- saslState = SaslState.COMPLETE;
- return KeeperState.SaslAuthenticated;
- }
- }
- }
- // No event ready to emit yet.
- return null;
- }
-
- // Initialize the client's communications with the Zookeeper server by sending the server the first
- // authentication packet.
- public void initialize(ClientCnxn cnxn) throws SaslException {
- if (saslClient == null) {
- saslState = SaslState.FAILED;
- throw new SaslException("saslClient failed to initialize properly: it's null.");
- }
- if (saslState == SaslState.INITIAL) {
- if (saslClient.hasInitialResponse()) {
- sendSaslPacket(cnxn);
- }
- else {
- byte[] emptyToken = new byte[0];
- sendSaslPacket(emptyToken, cnxn);
- }
- saslState = SaslState.INTERMEDIATE;
- }
- }
-
- public boolean clientTunneledAuthenticationInProgress() {
- if (!isSASLConfigured) {
- return false;
- }
- // TODO: Rather than checking a disjunction here, should be a single member
- // variable or method in this class to determine whether the client is
- // configured to use SASL. (see also ZOOKEEPER-1455).
- try {
- if ((clientConfig.getJaasConfKey() != null)
- || ((Configuration.getConfiguration() != null) && (Configuration.getConfiguration()
- .getAppConfigurationEntry(clientConfig.getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY,
- ZKClientConfig.LOGIN_CONTEXT_NAME_KEY_DEFAULT)) != null))) {
- // Client is configured to use a valid login Configuration, so
- // authentication is either in progress, successful, or failed.
-
- // 1. Authentication hasn't finished yet: we must wait for it to do so.
- if ((isComplete() == false) &&
- (isFailed() == false)) {
- return true;
- }
-
- // 2. SASL authentication has succeeded or failed..
- if (isComplete() || isFailed()) {
- if (gotLastPacket == false) {
- // ..but still in progress, because there is a final SASL
- // message from server which must be received.
- return true;
- }
- }
- }
- // Either client is not configured to use a tunnelled authentication
- // scheme, or tunnelled authentication has completed (successfully or
- // not), and all server SASL messages have been received.
- return false;
- } catch (SecurityException e) {
- // Thrown if the caller does not have permission to retrieve the Configuration.
- // In this case, simply returning false is correct.
- if (LOG.isDebugEnabled() == true) {
- LOG.debug("Could not retrieve login configuration: " + e);
- }
- return false;
- }
- }
-
- /**
- * close login thread if running
- */
- public void shutdown() {
- if (null != login) {
- login.shutdown();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/AtomicFileOutputStream.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/AtomicFileOutputStream.java b/src/java/main/org/apache/zookeeper/common/AtomicFileOutputStream.java
deleted file mode 100644
index 740ae8f..0000000
--- a/src/java/main/org/apache/zookeeper/common/AtomicFileOutputStream.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.zookeeper.common;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FilterOutputStream;
-import java.io.IOException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/*
- * This code is originally from HDFS, see the similarly named files there
- * in case of bug fixing, history, etc...
- */
-
-/**
- * A FileOutputStream that has the property that it will only show up at its
- * destination once it has been entirely written and flushed to disk. While
- * being written, it will use a .tmp suffix.
- *
- * When the output stream is closed, it is flushed, fsynced, and will be moved
- * into place, overwriting any file that already exists at that location.
- *
- * <b>NOTE</b>: on Windows platforms, it will not atomically replace the target
- * file - instead the target file is deleted before this one is moved into
- * place.
- */
-public class AtomicFileOutputStream extends FilterOutputStream {
- private static final String TMP_EXTENSION = ".tmp";
-
- private final static Logger LOG = LoggerFactory
- .getLogger(AtomicFileOutputStream.class);
-
- private final File origFile;
- private final File tmpFile;
-
- public AtomicFileOutputStream(File f) throws FileNotFoundException {
- // Code unfortunately must be duplicated below since we can't assign
- // anything
- // before calling super
- super(new FileOutputStream(new File(f.getParentFile(), f.getName()
- + TMP_EXTENSION)));
- origFile = f.getAbsoluteFile();
- tmpFile = new File(f.getParentFile(), f.getName() + TMP_EXTENSION)
- .getAbsoluteFile();
- }
-
- /**
- * The default write method in FilterOutputStream does not call the write
- * method of its underlying input stream with the same arguments. Instead
- * it writes the data byte by byte, override it here to make it more
- * efficient.
- */
- @Override
- public void write(byte b[], int off, int len) throws IOException {
- out.write(b, off, len);
- }
-
- @Override
- public void close() throws IOException {
- boolean triedToClose = false, success = false;
- try {
- flush();
- ((FileOutputStream) out).getFD().sync();
-
- triedToClose = true;
- super.close();
- success = true;
- } finally {
- if (success) {
- boolean renamed = tmpFile.renameTo(origFile);
- if (!renamed) {
- // On windows, renameTo does not replace.
- if (!origFile.delete() || !tmpFile.renameTo(origFile)) {
- throw new IOException(
- "Could not rename temporary file " + tmpFile
- + " to " + origFile);
- }
- }
- } else {
- if (!triedToClose) {
- // If we failed when flushing, try to close it to not leak
- // an FD
- IOUtils.closeStream(out);
- }
- // close wasn't successful, try to delete the tmp file
- if (!tmpFile.delete()) {
- LOG.warn("Unable to delete tmp file " + tmpFile);
- }
- }
- }
- }
-
- /**
- * Close the atomic file, but do not "commit" the temporary file on top of
- * the destination. This should be used if there is a failure in writing.
- */
- public void abort() {
- try {
- super.close();
- } catch (IOException ioe) {
- LOG.warn("Unable to abort file " + tmpFile, ioe);
- }
- if (!tmpFile.delete()) {
- LOG.warn("Unable to delete tmp file during abort " + tmpFile);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/AtomicFileWritingIdiom.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/AtomicFileWritingIdiom.java b/src/java/main/org/apache/zookeeper/common/AtomicFileWritingIdiom.java
deleted file mode 100644
index 441ea65..0000000
--- a/src/java/main/org/apache/zookeeper/common/AtomicFileWritingIdiom.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.zookeeper.common;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-
-/*
- * Used to perform an atomic write into a file.
- * If there is a failure in the middle of the writing operation,
- * the original file (if it exists) is left intact.
- * Based on the org.apache.zookeeper.server.quorum.QuorumPeer.writeLongToFile(...) idiom
- * using the HDFS AtomicFileOutputStream class.
- */
-public class AtomicFileWritingIdiom {
-
- public static interface OutputStreamStatement {
-
- public void write(OutputStream os) throws IOException;
-
- }
-
- public static interface WriterStatement {
-
- public void write(Writer os) throws IOException;
-
- }
-
- public AtomicFileWritingIdiom(File targetFile, OutputStreamStatement osStmt) throws IOException {
- this(targetFile, osStmt, null);
- }
-
- public AtomicFileWritingIdiom(File targetFile, WriterStatement wStmt) throws IOException {
- this(targetFile, null, wStmt);
- }
-
- private AtomicFileWritingIdiom(File targetFile, OutputStreamStatement osStmt, WriterStatement wStmt) throws IOException {
- AtomicFileOutputStream out = null;
- boolean error = true;
- try {
- out = new AtomicFileOutputStream(targetFile);
- if (wStmt == null) {
- // execute output stream operation
- osStmt.write(out);
- } else {
- BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out));
- // execute writer operation and flush
- wStmt.write(bw);
- bw.flush();
- }
- out.flush();
- // everything went ok
- error = false;
- } finally {
- // nothing interesting to do if out == null
- if (out != null) {
- if (error) {
- // worst case here the tmp file/resources(fd) are not cleaned up
- // and the caller will be notified (IOException)
- out.abort();
- } else {
- // if the close operation (rename) fails we'll get notified.
- // worst case the tmp file may still exist
- IOUtils.closeStream(out);
- }
- }
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/IOUtils.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/IOUtils.java b/src/java/main/org/apache/zookeeper/common/IOUtils.java
deleted file mode 100644
index 16aea4e..0000000
--- a/src/java/main/org/apache/zookeeper/common/IOUtils.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.zookeeper.common;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintStream;
-
-import org.slf4j.Logger;
-
-/*
- * This code is originally from HDFS, see the similarly named files there
- * in case of bug fixing, history, etc...
- */
-
-public class IOUtils {
- /**
- * Closes the stream ignoring {@link IOException}. Must only be called in
- * cleaning up from exception handlers.
- *
- * @param stream
- * the Stream to close
- */
- public static void closeStream(Closeable stream) {
- cleanup(null, stream);
- }
-
- /**
- * Close the Closeable objects and <b>ignore</b> any {@link IOException} or
- * null pointers. Must only be used for cleanup in exception handlers.
- *
- * @param log
- * the log to record problems to at debug level. Can be null.
- * @param closeables
- * the objects to close
- */
- public static void cleanup(Logger log, Closeable... closeables) {
- for (Closeable c : closeables) {
- if (c != null) {
- try {
- c.close();
- } catch (IOException e) {
- if (log != null) {
- log.warn("Exception in closing " + c, e);
- }
- }
- }
- }
- }
-
- /**
- * Copies from one stream to another.
- *
- * @param in
- * InputStrem to read from
- * @param out
- * OutputStream to write to
- * @param buffSize
- * the size of the buffer
- * @param close
- * whether or not close the InputStream and OutputStream at the
- * end. The streams are closed in the finally clause.
- */
- public static void copyBytes(InputStream in, OutputStream out,
- int buffSize, boolean close) throws IOException {
- try {
- copyBytes(in, out, buffSize);
- if (close) {
- out.close();
- out = null;
- in.close();
- in = null;
- }
- } finally {
- if (close) {
- closeStream(out);
- closeStream(in);
- }
- }
- }
-
- /**
- * Copies from one stream to another.
- *
- * @param in
- * InputStrem to read from
- * @param out
- * OutputStream to write to
- * @param buffSize
- * the size of the buffer
- */
- public static void copyBytes(InputStream in, OutputStream out, int buffSize)
- throws IOException {
- PrintStream ps = out instanceof PrintStream ? (PrintStream) out : null;
- byte buf[] = new byte[buffSize];
- int bytesRead = in.read(buf);
- while (bytesRead >= 0) {
- out.write(buf, 0, bytesRead);
- if ((ps != null) && ps.checkError()) {
- throw new IOException("Unable to write to output stream.");
- }
- bytesRead = in.read(buf);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/NetUtils.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/NetUtils.java b/src/java/main/org/apache/zookeeper/common/NetUtils.java
deleted file mode 100644
index 4779003..0000000
--- a/src/java/main/org/apache/zookeeper/common/NetUtils.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.common;
-
-import java.net.Inet6Address;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-
-/**
- * This class contains common utilities for netstuff. Like printing IPv6 literals correctly
- */
-public class NetUtils {
-
- public static String formatInetAddr(InetSocketAddress addr) {
- InetAddress ia = addr.getAddress();
-
- if (ia == null) {
- return String.format("%s:%s", addr.getHostString(), addr.getPort());
- }
-
- if (ia instanceof Inet6Address) {
- return String.format("[%s]:%s", ia.getHostAddress(), addr.getPort());
- } else {
- return String.format("%s:%s", ia.getHostAddress(), addr.getPort());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/PathTrie.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/PathTrie.java b/src/java/main/org/apache/zookeeper/common/PathTrie.java
deleted file mode 100644
index 5555b28..0000000
--- a/src/java/main/org/apache/zookeeper/common/PathTrie.java
+++ /dev/null
@@ -1,294 +0,0 @@
- /**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.common;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * a class that implements prefix matching for
- * components of a filesystem path. the trie
- * looks like a tree with edges mapping to
- * the component of a path.
- * example /ab/bc/cf would map to a trie
- * /
- * ab/
- * (ab)
- * bc/
- * /
- * (bc)
- * cf/
- * (cf)
- */
-public class PathTrie {
- /**
- * the logger for this class
- */
- private static final Logger LOG = LoggerFactory.getLogger(PathTrie.class);
-
- /**
- * the root node of PathTrie
- */
- private final TrieNode rootNode ;
-
- static class TrieNode {
- boolean property = false;
- final Map<String, TrieNode> children;
- TrieNode parent = null;
- /**
- * create a trienode with parent
- * as parameter
- * @param parent the parent of this trienode
- */
- private TrieNode(TrieNode parent) {
- children = new HashMap<String, TrieNode>();
- this.parent = parent;
- }
-
- /**
- * get the parent of this node
- * @return the parent node
- */
- TrieNode getParent() {
- return this.parent;
- }
-
- /**
- * set the parent of this node
- * @param parent the parent to set to
- */
- void setParent(TrieNode parent) {
- this.parent = parent;
- }
-
- /**
- * a property that is set
- * for a node - making it
- * special.
- */
- void setProperty(boolean prop) {
- this.property = prop;
- }
-
- /** the property of this
- * node
- * @return the property for this
- * node
- */
- boolean getProperty() {
- return this.property;
- }
- /**
- * add a child to the existing node
- * @param childName the string name of the child
- * @param node the node that is the child
- */
- void addChild(String childName, TrieNode node) {
- synchronized(children) {
- if (children.containsKey(childName)) {
- return;
- }
- children.put(childName, node);
- }
- }
-
- /**
- * delete child from this node
- * @param childName the string name of the child to
- * be deleted
- */
- void deleteChild(String childName) {
- synchronized(children) {
- if (!children.containsKey(childName)) {
- return;
- }
- TrieNode childNode = children.get(childName);
- // this is the only child node.
- if (childNode.getChildren().length == 1) {
- childNode.setParent(null);
- children.remove(childName);
- }
- else {
- // their are more child nodes
- // so just reset property.
- childNode.setProperty(false);
- }
- }
- }
-
- /**
- * return the child of a node mapping
- * to the input childname
- * @param childName the name of the child
- * @return the child of a node
- */
- TrieNode getChild(String childName) {
- synchronized(children) {
- if (!children.containsKey(childName)) {
- return null;
- }
- else {
- return children.get(childName);
- }
- }
- }
-
- /**
- * get the list of children of this
- * trienode.
- * @param node to get its children
- * @return the string list of its children
- */
- String[] getChildren() {
- synchronized(children) {
- return children.keySet().toArray(new String[0]);
- }
- }
-
- /**
- * get the string representation
- * for this node
- */
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("Children of trienode: ");
- synchronized(children) {
- for (String str: children.keySet()) {
- sb.append(" " + str);
- }
- }
- return sb.toString();
- }
- }
-
- /**
- * construct a new PathTrie with
- * a root node of /
- */
- public PathTrie() {
- this.rootNode = new TrieNode(null);
- }
-
- /**
- * add a path to the path trie
- * @param path
- */
- public void addPath(String path) {
- if (path == null) {
- return;
- }
- String[] pathComponents = path.split("/");
- TrieNode parent = rootNode;
- String part = null;
- if (pathComponents.length <= 1) {
- throw new IllegalArgumentException("Invalid path " + path);
- }
- for (int i=1; i<pathComponents.length; i++) {
- part = pathComponents[i];
- if (parent.getChild(part) == null) {
- parent.addChild(part, new TrieNode(parent));
- }
- parent = parent.getChild(part);
- }
- parent.setProperty(true);
- }
-
- /**
- * delete a path from the trie
- * @param path the path to be deleted
- */
- public void deletePath(String path) {
- if (path == null) {
- return;
- }
- String[] pathComponents = path.split("/");
- TrieNode parent = rootNode;
- String part = null;
- if (pathComponents.length <= 1) {
- throw new IllegalArgumentException("Invalid path " + path);
- }
- for (int i=1; i<pathComponents.length; i++) {
- part = pathComponents[i];
- if (parent.getChild(part) == null) {
- //the path does not exist
- return;
- }
- parent = parent.getChild(part);
- LOG.info("{}",parent);
- }
- TrieNode realParent = parent.getParent();
- realParent.deleteChild(part);
- }
-
- /**
- * return the largest prefix for the input path.
- * @param path the input path
- * @return the largest prefix for the input path.
- */
- public String findMaxPrefix(String path) {
- if (path == null) {
- return null;
- }
- if ("/".equals(path)) {
- return path;
- }
- String[] pathComponents = path.split("/");
- TrieNode parent = rootNode;
- List<String> components = new ArrayList<String>();
- if (pathComponents.length <= 1) {
- throw new IllegalArgumentException("Invalid path " + path);
- }
- int i = 1;
- String part = null;
- StringBuilder sb = new StringBuilder();
- int lastindex = -1;
- while((i < pathComponents.length)) {
- if (parent.getChild(pathComponents[i]) != null) {
- part = pathComponents[i];
- parent = parent.getChild(part);
- components.add(part);
- if (parent.getProperty()) {
- lastindex = i-1;
- }
- }
- else {
- break;
- }
- i++;
- }
- for (int j=0; j< (lastindex+1); j++) {
- sb.append("/" + components.get(j));
- }
- return sb.toString();
- }
-
- /**
- * clear all nodes
- */
- public void clear() {
- for(String child : rootNode.getChildren()) {
- rootNode.deleteChild(child);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/PathUtils.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/PathUtils.java b/src/java/main/org/apache/zookeeper/common/PathUtils.java
deleted file mode 100644
index d80feab..0000000
--- a/src/java/main/org/apache/zookeeper/common/PathUtils.java
+++ /dev/null
@@ -1,120 +0,0 @@
- /**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.common;
-
-
-/**
- * Path related utilities
- */
-public class PathUtils {
-
- /** validate the provided znode path string
- * @param path znode path string
- * @param isSequential if the path is being created
- * with a sequential flag
- * @throws IllegalArgumentException if the path is invalid
- */
- public static void validatePath(String path, boolean isSequential)
- throws IllegalArgumentException {
- validatePath(isSequential? path + "1": path);
- }
-
- /**
- * Validate the provided znode path string
- * @param path znode path string
- * @throws IllegalArgumentException if the path is invalid
- */
- public static void validatePath(String path) throws IllegalArgumentException {
- if (path == null) {
- throw new IllegalArgumentException("Path cannot be null");
- }
- if (path.length() == 0) {
- throw new IllegalArgumentException("Path length must be > 0");
- }
- if (path.charAt(0) != '/') {
- throw new IllegalArgumentException(
- "Path must start with / character");
- }
- if (path.length() == 1) { // done checking - it's the root
- return;
- }
- if (path.charAt(path.length() - 1) == '/') {
- throw new IllegalArgumentException(
- "Path must not end with / character");
- }
-
- String reason = null;
- char lastc = '/';
- char chars[] = path.toCharArray();
- char c;
- for (int i = 1; i < chars.length; lastc = chars[i], i++) {
- c = chars[i];
-
- if (c == 0) {
- reason = "null character not allowed @" + i;
- break;
- } else if (c == '/' && lastc == '/') {
- reason = "empty node name specified @" + i;
- break;
- } else if (c == '.' && lastc == '.') {
- if (chars[i-2] == '/' &&
- ((i + 1 == chars.length)
- || chars[i+1] == '/')) {
- reason = "relative paths not allowed @" + i;
- break;
- }
- } else if (c == '.') {
- if (chars[i-1] == '/' &&
- ((i + 1 == chars.length)
- || chars[i+1] == '/')) {
- reason = "relative paths not allowed @" + i;
- break;
- }
- } else if (c > '\u0000' && c <= '\u001f'
- || c >= '\u007f' && c <= '\u009F'
- || c >= '\ud800' && c <= '\uf8ff'
- || c >= '\ufff0' && c <= '\uffff') {
- reason = "invalid character @" + i;
- break;
- }
- }
-
- if (reason != null) {
- throw new IllegalArgumentException(
- "Invalid path string \"" + path + "\" caused by " + reason);
- }
- }
-
- /**
- * Convert Windows path to Unix
- *
- * @param path
- * file path
- * @return converted file path
- */
- public static String normalizeFileSystemPath(String path) {
- if (path != null) {
- String osname = java.lang.System.getProperty("os.name");
- if (osname.toLowerCase().contains("windows")) {
- return path.replace('\\', '/');
- }
- }
- return path;
- }
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/StringUtils.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/StringUtils.java b/src/java/main/org/apache/zookeeper/common/StringUtils.java
deleted file mode 100644
index 9e0f1a9..0000000
--- a/src/java/main/org/apache/zookeeper/common/StringUtils.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/* Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.common;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Collections;
-
-public class StringUtils {
-
- private StringUtils() {/** non instantiable and non inheritable **/}
-
- /**
- * This method returns an immutable List<String>, but different from String's split()
- * it trims the results in the input String, and removes any empty string from
- * the resulting List.
- *
- */
- public static List<String> split(String value, String separator) {
- String[] splits = value.split(separator);
- List<String> results = new ArrayList<String>();
- for (int i = 0; i < splits.length; i++) {
- splits[i] = splits[i].trim();
- if (splits[i].length() > 0) {
- results.add(splits[i]);
- }
- }
- return Collections.unmodifiableList(results);
- }
-
- /**
- * This method takes a List<String> and a delimiter and joins the strings
- * into a single string, where the original strings are separated using
- * the given delimiter.
- *
- */
- public static String joinStrings(List<String> list, String delim)
- {
- if (list == null)
- return null;
-
- StringBuilder builder = new StringBuilder(list.get(0));
- for (String s : list.subList(1, list.size())) {
- builder.append(delim).append(s);
- }
-
- return builder.toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/Time.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/Time.java b/src/java/main/org/apache/zookeeper/common/Time.java
deleted file mode 100644
index 83e53f0..0000000
--- a/src/java/main/org/apache/zookeeper/common/Time.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.common;
-
-import java.util.Date;
-
-public class Time {
- /**
- * Returns time in milliseconds as does System.currentTimeMillis(),
- * but uses elapsed time from an arbitrary epoch more like System.nanoTime().
- * The difference is that if somebody changes the system clock,
- * Time.currentElapsedTime will change but nanoTime won't. On the other hand,
- * all of ZK assumes that time is measured in milliseconds.
- * @return The time in milliseconds from some arbitrary point in time.
- */
- public static long currentElapsedTime() {
- return System.nanoTime() / 1000000;
- }
-
- /**
- * Explicitly returns system dependent current wall time.
- * @return Current time in msec.
- */
- public static long currentWallTime() {
- return System.currentTimeMillis();
- }
-
- /**
- * This is to convert the elapsedTime to a Date.
- * @return A date object indicated by the elapsedTime.
- */
- public static Date elapsedTimeToDate(long elapsedTime) {
- long wallTime = currentWallTime() + elapsedTime - currentElapsedTime();
- return new Date(wallTime);
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/X509Exception.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/X509Exception.java b/src/java/main/org/apache/zookeeper/common/X509Exception.java
deleted file mode 100644
index 984a2ab..0000000
--- a/src/java/main/org/apache/zookeeper/common/X509Exception.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.common;
-
-@SuppressWarnings("serial")
-public class X509Exception extends Exception {
- public X509Exception(String message) {
- super(message);
- }
-
- public X509Exception(Throwable cause) {
- super(cause);
- }
-
- public X509Exception(String message, Throwable cause) {
- super(message, cause);
- }
-
- public static class KeyManagerException extends X509Exception {
- public KeyManagerException(String message) {
- super(message);
- }
-
- public KeyManagerException(Throwable cause) {
- super(cause);
- }
- }
-
- public static class TrustManagerException extends X509Exception {
- public TrustManagerException(String message) {
- super(message);
- }
-
- public TrustManagerException(Throwable cause) {
- super(cause);
- }
- }
-
- public static class SSLContextException extends X509Exception {
- public SSLContextException(String message) {
- super(message);
- }
-
- public SSLContextException(Throwable cause) {
- super(cause);
- }
-
- public SSLContextException(String message, Throwable cause) {
- super(message, cause);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/X509Util.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/X509Util.java b/src/java/main/org/apache/zookeeper/common/X509Util.java
deleted file mode 100644
index cc8662e..0000000
--- a/src/java/main/org/apache/zookeeper/common/X509Util.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.zookeeper.common;
-
-
-import javax.net.ssl.KeyManager;
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.TrustManagerFactory;
-import javax.net.ssl.X509KeyManager;
-import javax.net.ssl.X509TrustManager;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.security.KeyStore;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.zookeeper.common.X509Exception.KeyManagerException;
-import static org.apache.zookeeper.common.X509Exception.SSLContextException;
-import static org.apache.zookeeper.common.X509Exception.TrustManagerException;
-
-/**
- * Utility code for X509 handling
- */
-public class X509Util {
- private static final Logger LOG = LoggerFactory.getLogger(X509Util.class);
-
- /**
- * @deprecated Use {@link ZKConfig#SSL_KEYSTORE_LOCATION}
- * instead.
- */
- @Deprecated
- public static final String SSL_KEYSTORE_LOCATION = "zookeeper.ssl.keyStore.location";
- /**
- * @deprecated Use {@link ZKConfig#SSL_KEYSTORE_PASSWD}
- * instead.
- */
- @Deprecated
- public static final String SSL_KEYSTORE_PASSWD = "zookeeper.ssl.keyStore.password";
- /**
- * @deprecated Use {@link ZKConfig#SSL_TRUSTSTORE_LOCATION}
- * instead.
- */
- @Deprecated
- public static final String SSL_TRUSTSTORE_LOCATION = "zookeeper.ssl.trustStore.location";
- /**
- * @deprecated Use {@link ZKConfig#SSL_TRUSTSTORE_PASSWD}
- * instead.
- */
- @Deprecated
- public static final String SSL_TRUSTSTORE_PASSWD = "zookeeper.ssl.trustStore.password";
- /**
- * @deprecated Use {@link ZKConfig#SSL_AUTHPROVIDER}
- * instead.
- */
- @Deprecated
- public static final String SSL_AUTHPROVIDER = "zookeeper.ssl.authProvider";
-
- public static SSLContext createSSLContext() throws SSLContextException {
- /**
- * Since Configuration initializes the key store and trust store related
- * configuration from system property. Reading property from
- * configuration will be same reading from system property
- */
- ZKConfig config=new ZKConfig();
- return createSSLContext(config);
- }
-
- public static SSLContext createSSLContext(ZKConfig config) throws SSLContextException {
- KeyManager[] keyManagers = null;
- TrustManager[] trustManagers = null;
-
- String keyStoreLocationProp = config.getProperty(ZKConfig.SSL_KEYSTORE_LOCATION);
- String keyStorePasswordProp = config.getProperty(ZKConfig.SSL_KEYSTORE_PASSWD);
-
- // There are legal states in some use cases for null KeyManager or TrustManager.
- // But if a user wanna specify one, location and password are required.
-
- if (keyStoreLocationProp == null && keyStorePasswordProp == null) {
- LOG.warn("keystore not specified for client connection");
- } else {
- if (keyStoreLocationProp == null) {
- throw new SSLContextException("keystore location not specified for client connection");
- }
- if (keyStorePasswordProp == null) {
- throw new SSLContextException("keystore password not specified for client connection");
- }
- try {
- keyManagers = new KeyManager[]{
- createKeyManager(keyStoreLocationProp, keyStorePasswordProp)};
- } catch (KeyManagerException e) {
- throw new SSLContextException("Failed to create KeyManager", e);
- }
- }
-
- String trustStoreLocationProp = config.getProperty(ZKConfig.SSL_TRUSTSTORE_LOCATION);
- String trustStorePasswordProp = config.getProperty(ZKConfig.SSL_TRUSTSTORE_PASSWD);
-
- if (trustStoreLocationProp == null && trustStorePasswordProp == null) {
- LOG.warn("Truststore not specified for client connection");
- } else {
- if (trustStoreLocationProp == null) {
- throw new SSLContextException("Truststore location not specified for client connection");
- }
- if (trustStorePasswordProp == null) {
- throw new SSLContextException("Truststore password not specified for client connection");
- }
- try {
- trustManagers = new TrustManager[]{
- createTrustManager(trustStoreLocationProp, trustStorePasswordProp)};
- } catch (TrustManagerException e) {
- throw new SSLContextException("Failed to create TrustManager", e);
- }
- }
-
- SSLContext sslContext = null;
- try {
- sslContext = SSLContext.getInstance("TLSv1");
- sslContext.init(keyManagers, trustManagers, null);
- } catch (Exception e) {
- throw new SSLContextException(e);
- }
- return sslContext;
- }
-
- public static X509KeyManager createKeyManager(String keyStoreLocation, String keyStorePassword)
- throws KeyManagerException {
- FileInputStream inputStream = null;
- try {
- char[] keyStorePasswordChars = keyStorePassword.toCharArray();
- File keyStoreFile = new File(keyStoreLocation);
- KeyStore ks = KeyStore.getInstance("JKS");
- inputStream = new FileInputStream(keyStoreFile);
- ks.load(inputStream, keyStorePasswordChars);
- KeyManagerFactory kmf = KeyManagerFactory.getInstance("SunX509");
- kmf.init(ks, keyStorePasswordChars);
-
- for (KeyManager km : kmf.getKeyManagers()) {
- if (km instanceof X509KeyManager) {
- return (X509KeyManager) km;
- }
- }
- throw new KeyManagerException("Couldn't find X509KeyManager");
-
- } catch (Exception e) {
- throw new KeyManagerException(e);
- } finally {
- if (inputStream != null) {
- try {
- inputStream.close();
- } catch (IOException e) {}
- }
- }
- }
-
- public static X509TrustManager createTrustManager(String trustStoreLocation, String trustStorePassword)
- throws TrustManagerException {
- FileInputStream inputStream = null;
- try {
- char[] trustStorePasswordChars = trustStorePassword.toCharArray();
- File trustStoreFile = new File(trustStoreLocation);
- KeyStore ts = KeyStore.getInstance("JKS");
- inputStream = new FileInputStream(trustStoreFile);
- ts.load(inputStream, trustStorePasswordChars);
- TrustManagerFactory tmf = TrustManagerFactory.getInstance("SunX509");
- tmf.init(ts);
-
- for (TrustManager tm : tmf.getTrustManagers()) {
- if (tm instanceof X509TrustManager) {
- return (X509TrustManager) tm;
- }
- }
- throw new TrustManagerException("Couldn't find X509TrustManager");
- } catch (Exception e) {
- throw new TrustManagerException(e);
- } finally {
- if (inputStream != null) {
- try {
- inputStream.close();
- } catch (IOException e) {}
- }
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/common/ZKConfig.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/ZKConfig.java b/src/java/main/org/apache/zookeeper/common/ZKConfig.java
deleted file mode 100644
index 8d9c001..0000000
--- a/src/java/main/org/apache/zookeeper/common/ZKConfig.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.common;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-
-import org.apache.zookeeper.Environment;
-import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
-import org.apache.zookeeper.server.util.VerifyingFileFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is a base class for the configurations of both client and server.
- * It supports reading client configuration from both system properties and
- * configuration file. A user can override any system property by calling
- * {@link #setProperty(String, String)}.
- * @since 3.5.2
- */
-public class ZKConfig {
-
- private static final Logger LOG = LoggerFactory.getLogger(ZKConfig.class);
- @SuppressWarnings("deprecation")
- public static final String SSL_KEYSTORE_LOCATION = X509Util.SSL_KEYSTORE_LOCATION;
- @SuppressWarnings("deprecation")
- public static final String SSL_KEYSTORE_PASSWD = X509Util.SSL_KEYSTORE_PASSWD;
- @SuppressWarnings("deprecation")
- public static final String SSL_TRUSTSTORE_LOCATION = X509Util.SSL_TRUSTSTORE_LOCATION;
- @SuppressWarnings("deprecation")
- public static final String SSL_TRUSTSTORE_PASSWD = X509Util.SSL_TRUSTSTORE_PASSWD;
- @SuppressWarnings("deprecation")
- public static final String SSL_AUTHPROVIDER = X509Util.SSL_AUTHPROVIDER;
- public static final String JUTE_MAXBUFFER = "jute.maxbuffer";
- /**
- * Path to a kinit binary: {@value}. Defaults to
- * <code>"/usr/bin/kinit"</code>
- */
- public static final String KINIT_COMMAND = "zookeeper.kinit";
- public static final String JGSS_NATIVE = "sun.security.jgss.native";
-
- private final Map<String, String> properties = new HashMap<String, String>();
-
- /**
- * properties, which are common to both client and server, are initialized
- * from system properties
- */
- public ZKConfig() {
- init();
- }
-
- /**
- * @param configPath
- * Configuration file path
- * @throws ConfigException
- * if failed to load configuration properties
- */
-
- public ZKConfig(String configPath) throws ConfigException {
- this(new File(configPath));
- }
-
- /**
- *
- * @param configFile
- * Configuration file
- * @throws ConfigException
- * if failed to load configuration properties
- */
- public ZKConfig(File configFile) throws ConfigException {
- this();
- addConfiguration(configFile);
- }
-
- private void init() {
- /**
- * backward compatibility for all currently available client properties
- */
- handleBackwardCompatibility();
- }
-
- /**
- * Now onwards client code will use properties from this class but older
- * clients still be setting properties through system properties. So to make
- * this change backward compatible we should set old system properties in
- * this configuration.
- */
- protected void handleBackwardCompatibility() {
- properties.put(SSL_KEYSTORE_LOCATION, System.getProperty(SSL_KEYSTORE_LOCATION));
- properties.put(SSL_KEYSTORE_PASSWD, System.getProperty(SSL_KEYSTORE_PASSWD));
- properties.put(SSL_TRUSTSTORE_LOCATION, System.getProperty(SSL_TRUSTSTORE_LOCATION));
- properties.put(SSL_TRUSTSTORE_PASSWD, System.getProperty(SSL_TRUSTSTORE_PASSWD));
- properties.put(SSL_AUTHPROVIDER, System.getProperty(SSL_AUTHPROVIDER));
- properties.put(JUTE_MAXBUFFER, System.getProperty(JUTE_MAXBUFFER));
- properties.put(KINIT_COMMAND, System.getProperty(KINIT_COMMAND));
- properties.put(JGSS_NATIVE, System.getProperty(JGSS_NATIVE));
- }
-
- /**
- * Get the property value
- *
- * @param key
- * @return property value
- */
- public String getProperty(String key) {
- return properties.get(key);
- }
-
- /**
- * Get the property value, if it is null return default value
- *
- * @param key
- * property key
- * @param defaultValue
- * @return property value or default value
- */
- public String getProperty(String key, String defaultValue) {
- String value = properties.get(key);
- return (value == null) ? defaultValue : value;
- }
-
- /**
- * Return the value of "java.security.auth.login.config" system property
- *
- * @return value
- */
- public String getJaasConfKey() {
- return System.getProperty(Environment.JAAS_CONF_KEY);
- }
-
- /**
- * Maps the specified <code>key</code> to the specified <code>value</code>.
- * key can not be <code>null</code>. If key is already mapped then the old
- * value of the <code>key</code> is replaced by the specified
- * <code>value</code>.
- *
- * @param key
- * @param value
- */
- public void setProperty(String key, String value) {
- if (null == key) {
- throw new IllegalArgumentException("property key is null.");
- }
- String oldValue = properties.put(key, value);
- if (LOG.isDebugEnabled()) {
- if (null != oldValue && !oldValue.equals(value)) {
- LOG.debug("key {}'s value {} is replaced with new value {}", key, oldValue, value);
- }
- }
- }
-
- /**
- * Add a configuration resource. The properties form this configuration will
- * overwrite corresponding already loaded property and system property
- *
- * @param configFile
- * Configuration file.
- */
- public void addConfiguration(File configFile) throws ConfigException {
- LOG.info("Reading configuration from: {}", configFile.getAbsolutePath());
- try {
- configFile = (new VerifyingFileFactory.Builder(LOG).warnForRelativePath().failForNonExistingPath().build())
- .validate(configFile);
- Properties cfg = new Properties();
- FileInputStream in = new FileInputStream(configFile);
- try {
- cfg.load(in);
- } finally {
- in.close();
- }
- parseProperties(cfg);
- } catch (IOException | IllegalArgumentException e) {
- LOG.error("Error while configuration from: {}", configFile.getAbsolutePath(), e);
- throw new ConfigException("Error while processing " + configFile.getAbsolutePath(), e);
- }
- }
-
- /**
- * Add a configuration resource. The properties form this configuration will
- * overwrite corresponding already loaded property and system property
- *
- * @param configPath
- * Configuration file path.
- */
- public void addConfiguration(String configPath) throws ConfigException {
- addConfiguration(new File(configPath));
- }
-
- private void parseProperties(Properties cfg) {
- for (Entry<Object, Object> entry : cfg.entrySet()) {
- String key = entry.getKey().toString().trim();
- String value = entry.getValue().toString().trim();
- setProperty(key, value);
- }
- }
-
- /**
- * Returns {@code true} if and only if the property named by the argument
- * exists and is equal to the string {@code "true"}.
- */
- public boolean getBoolean(String key) {
- return Boolean.parseBoolean(getProperty(key));
- }
-
- /**
- * Get the value of the <code>key</code> property as an <code>int</code>. If
- * property is not set, the provided <code>defaultValue</code> is returned
- *
- * @param key
- * property key.
- * @param defaultValue
- * default value.
- * @throws NumberFormatException
- * when the value is invalid
- * @return return property value as an <code>int</code>, or
- * <code>defaultValue</code>
- */
- public int getInt(String key, int defaultValue) {
- String value = getProperty(key);
- if (value != null) {
- return Integer.parseInt(value.trim());
- }
- return defaultValue;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/jmx/CommonNames.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/jmx/CommonNames.java b/src/java/main/org/apache/zookeeper/jmx/CommonNames.java
deleted file mode 100644
index 986d69d..0000000
--- a/src/java/main/org/apache/zookeeper/jmx/CommonNames.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.jmx;
-
-/**
- * A bunch of constants.
- * TODO: will get rid of it eventually.
- */
-public class CommonNames {
- public static final String DOMAIN="org.apache.ZooKeeperService";
- public static final String DATA_TREE_KEY="DataTree";
- public static final String STANDALONE_SERVER_KEY="StandaloneServer";
-}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/faa7cec7/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java b/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java
deleted file mode 100644
index 817afc0..0000000
--- a/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.zookeeper.jmx;
-
-import java.lang.management.ManagementFactory;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.management.JMException;
-import javax.management.MBeanServer;
-import javax.management.MBeanServerFactory;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides a unified interface for registering/unregistering of
- * zookeeper MBeans with the platform MBean server. It builds a hierarchy of MBeans
- * where each MBean represented by a filesystem-like path. Eventually, this hierarchy
- * will be stored in the zookeeper data tree instance as a virtual data tree.
- */
-public class MBeanRegistry {
- private static final Logger LOG = LoggerFactory.getLogger(MBeanRegistry.class);
-
- private static volatile MBeanRegistry instance = new MBeanRegistry();
-
- private final Object LOCK = new Object();
-
- private Map<ZKMBeanInfo, String> mapBean2Path =
- new ConcurrentHashMap<ZKMBeanInfo, String>();
-
- private MBeanServer mBeanServer;
-
- /**
- * Useful for unit tests. Change the MBeanRegistry instance
- *
- * @param instance new instance
- */
- public static void setInstance(MBeanRegistry instance) {
- MBeanRegistry.instance = instance;
- }
-
- public static MBeanRegistry getInstance() {
- return instance;
- }
-
- public MBeanRegistry () {
- try {
- mBeanServer = ManagementFactory.getPlatformMBeanServer();
- } catch (Error e) {
- // Account for running within IKVM and create a new MBeanServer
- // if the PlatformMBeanServer does not exist.
- mBeanServer = MBeanServerFactory.createMBeanServer();
- }
- }
-
- /**
- * Return the underlying MBeanServer that is being
- * used to register MBean's. The returned MBeanServer
- * may be a new empty MBeanServer if running through IKVM.
- */
- public MBeanServer getPlatformMBeanServer() {
- return mBeanServer;
- }
-
- /**
- * Registers a new MBean with the platform MBean server.
- * @param bean the bean being registered
- * @param parent if not null, the new bean will be registered as a child
- * node of this parent.
- */
- public void register(ZKMBeanInfo bean, ZKMBeanInfo parent)
- throws JMException
- {
- assert bean != null;
- String path = null;
- if (parent != null) {
- path = mapBean2Path.get(parent);
- assert path != null;
- }
- path = makeFullPath(path, parent);
- if(bean.isHidden())
- return;
- ObjectName oname = makeObjectName(path, bean);
- try {
- synchronized (LOCK) {
- mBeanServer.registerMBean(bean, oname);
- mapBean2Path.put(bean, path);
- }
- } catch (JMException e) {
- LOG.warn("Failed to register MBean " + bean.getName());
- throw e;
- }
- }
-
- /**
- * Unregister the MBean identified by the path.
- * @param path
- * @param bean
- */
- private void unregister(String path,ZKMBeanInfo bean) throws JMException {
- if(path==null)
- return;
- if (!bean.isHidden()) {
- final ObjectName objName = makeObjectName(path, bean);
- LOG.debug("Unregister MBean [{}]", objName);
- synchronized (LOCK) {
- mBeanServer.unregisterMBean(objName);
- }
- }
- }
-
- /**
- * @return a {@link Collection} with the {@link ZKMBeanInfo} instances not
- * unregistered. Mainly for testing purposes.
- */
- public Set<ZKMBeanInfo> getRegisteredBeans() {
- return new HashSet<ZKMBeanInfo>(mapBean2Path.keySet());
- }
-
- /**
- * Unregister MBean.
- * @param bean
- */
- public void unregister(ZKMBeanInfo bean) {
- if(bean==null)
- return;
- String path = mapBean2Path.remove(bean);
- try {
- unregister(path,bean);
- } catch (JMException e) {
- LOG.warn("Error during unregister of [{}]", bean.getName(), e);
- } catch (Throwable t) {
- LOG.error("Unexpected exception during unregister of [{}]. It should be reviewed and fixed.", bean.getName(), t);
- }
- }
-
- /**
- * Generate a filesystem-like path.
- * @param prefix path prefix
- * @param name path elements
- * @return absolute path
- */
- public String makeFullPath(String prefix, String... name) {
- StringBuilder sb=new StringBuilder(prefix == null ? "/" : (prefix.equals("/")?prefix:prefix+"/"));
- boolean first=true;
- for (String s : name) {
- if(s==null) continue;
- if(!first){
- sb.append("/");
- }else
- first=false;
- sb.append(s);
- }
- return sb.toString();
- }
-
- protected String makeFullPath(String prefix, ZKMBeanInfo bean) {
- return makeFullPath(prefix, bean == null ? null : bean.getName());
- }
-
- /**
- * This takes a path, such as /a/b/c, and converts it to
- * name0=a,name1=b,name2=c
- */
- private int tokenize(StringBuilder sb, String path, int index){
- String[] tokens = path.split("/");
- for (String s: tokens) {
- if (s.length()==0)
- continue;
- sb.append("name").append(index++)
- .append("=").append(s).append(",");
- }
- return index;
- }
- /**
- * Builds an MBean path and creates an ObjectName instance using the path.
- * @param path MBean path
- * @param bean the MBean instance
- * @return ObjectName to be registered with the platform MBean server
- */
- protected ObjectName makeObjectName(String path, ZKMBeanInfo bean)
- throws MalformedObjectNameException
- {
- if(path==null)
- return null;
- StringBuilder beanName = new StringBuilder(CommonNames.DOMAIN + ":");
- int counter=0;
- counter=tokenize(beanName,path,counter);
- tokenize(beanName,bean.getName(),counter);
- beanName.deleteCharAt(beanName.length()-1);
- try {
- return new ObjectName(beanName.toString());
- } catch (MalformedObjectNameException e) {
- LOG.warn("Invalid name \"" + beanName.toString() + "\" for class "
- + bean.getClass().toString());
- throw e;
- }
- }
-}
|