http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.cc
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.cc b/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.cc
new file mode 100644
index 0000000..270bf31
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.cc
@@ -0,0 +1,705 @@
+/**
+ * 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.
+ */
+
+#include "ZkTreeUtil.h"
+
+#include <map>
+#include <iostream>
+#include <log4cxx/logger.h>
+#include <boost/algorithm/string.hpp>
+#include <boost/algorithm/string/split.hpp>
+
+namespace zktreeutil
+{
+ using std::map;
+ using std::pair;
+
+ static ZkTreeNodeSptr loadZkTree_ (ZooKeeperAdapterSptr zkHandle,
+ const string& path)
+ {
+ // Extract the node value
+ string value = zkHandle->getNodeData(path);
+
+ // Extract nodename from the path
+ string nodename = "/";
+ if (path != "/")
+ {
+ vector< string > nodes;
+ boost::split(nodes, path, boost::is_any_of ("/") );
+ nodename = nodes[nodes.size()-1];
+ }
+
+ // Create tree-node with name and value
+ ZkTreeNodeSptr nodeSptr = ZkTreeNodeSptr (new ZkTreeNode (nodename, value));
+ std::cerr << "[zktreeutil] loaded nodename: "
+ << nodename
+ << " value: "
+ << value
+ << std::endl;
+
+ // Load all the children
+ vector< string > cnodes = zkHandle->getNodeChildren (path);
+ for (unsigned i = 0; i < cnodes.size(); i++)
+ nodeSptr->addChild (loadZkTree_ (zkHandle, cnodes[i]));
+
+ // Return the constructed node
+ return nodeSptr;
+ }
+
+ static ZkTreeNodeSptr loadZkTreeXml_ (xmlNode* xmlNodePtr)
+ {
+ // Null check
+ if (xmlNodePtr == NULL)
+ {
+ std::cerr << "[zktreeutil] empty XML node encountered" << std::endl;
+ exit (-1);
+ }
+
+ // Get the node name
+ xmlChar* name = xmlGetProp (xmlNodePtr, BAD_CAST "name");
+ string nameStr = (const char*)name;
+ std::cerr << "[zktreeutil] node name: " << nameStr;
+ xmlFree (name);
+ // Get the node value
+ string valueStr;
+ xmlChar* value = xmlGetProp (xmlNodePtr, BAD_CAST "value");
+ if (value)
+ {
+ valueStr = (const char*)value;
+ std::cerr << " value: " << valueStr;
+ }
+ xmlFree (value);
+ // Get the ignore flag
+ bool doIgnore = false;
+ xmlChar* ignore = xmlGetProp (xmlNodePtr, BAD_CAST "ignore");
+ if (ignore)
+ {
+ string ignoreStr = (const char*) ignore;
+ if (ignoreStr == "true" || ignoreStr == "yes" || ignoreStr == "1")
+ {
+ doIgnore = true;
+ std::cerr << " <ignore:>";
+ }
+ }
+ xmlFree (ignore);
+ std::cerr << std::endl;
+
+ // Create the zk node
+ ZkTreeNodeSptr nodeSptr =
+ ZkTreeNodeSptr (new ZkTreeNode (nameStr,
+ ZkNodeData (valueStr, doIgnore)));
+
+ // Load the children
+ for (xmlNode* chldNode = xmlNodePtr->children;
+ chldNode;
+ chldNode = chldNode->next)
+ if (chldNode->type == XML_ELEMENT_NODE)
+ nodeSptr->addChild (loadZkTreeXml_ (chldNode));
+
+ // Return the loaded node
+ return nodeSptr;
+ }
+
+ static void writeZkTree_ (ZooKeeperAdapterSptr zkHandle,
+ const ZkTreeNodeSptr zkNodeSptr,
+ const string& path)
+ {
+ // Create the path in zk-tree
+ zkHandle->createNode(path.c_str(), "", 0, false);
+ std::cerr << "[zktreeutil] created key: " << path << std::endl;
+ // Set value for the path
+ string value = zkNodeSptr->getData().value;
+ if (value != "")
+ {
+ zkHandle->setNodeData (path.c_str(), value.c_str());
+ std::cerr << "[zktreeutil] set value: " << std::endl;
+ }
+
+ // Go deep to write the subtree rooted in the node, if not to be ignored
+ if (!(zkNodeSptr->getData().ignoreUpdate))
+ {
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childNodeSptr = zkNodeSptr->getChild (i);
+ // Add the node name into the path and write in zk-tree
+ string cpath = ((path != "/")? path : "")
+ + string("/")
+ + childNodeSptr->getKey();
+ writeZkTree_ (zkHandle, childNodeSptr, cpath);
+ }
+ }
+
+ return;
+ }
+
+ static void addTreeZkAction_ (const ZkTreeNodeSptr zkNodeSptr,
+ const string& path,
+ vector< ZkAction >& actions)
+ {
+ // Create the key
+ actions.push_back (ZkAction (ZkAction::CREATE, path));
+
+ // Set value for the new key
+ if (zkNodeSptr->getData().value != "")
+ actions.push_back (ZkAction (ZkAction::VALUE,
+ path,
+ zkNodeSptr->getData().value));
+
+ // Add all the children
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childSptr = zkNodeSptr->getChild (i);
+ string cpath = path + string("/") + childSptr->getKey();
+ addTreeZkAction_ (childSptr, cpath, actions);
+ }
+
+ return;
+ }
+
+ static xmlNodePtr dumpZkTreeXml_ (const ZkTreeNodeSptr zkNodeSptr)
+ {
+ // Create xml node with zknode name and value
+ string nodename = zkNodeSptr->getKey ();
+ string value = zkNodeSptr->getData().value;
+ xmlNodePtr node = xmlNewNode(NULL, BAD_CAST "zknode");
+ xmlNewProp (node, BAD_CAST "name", BAD_CAST nodename.c_str());
+ if (value.length())
+ xmlNewProp (node, BAD_CAST "value", BAD_CAST value.c_str());
+
+ // Add all the children rotted at this node
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ xmlAddChild (node, dumpZkTreeXml_ (zkNodeSptr->getChild (i)));
+
+ // Return xml node
+ return node;
+ }
+
+ static void dumpZkTree_ (const ZkTreeNodeSptr zkNodeSptr,
+ int maxLevel,
+ int level,
+ vector< bool >& masks)
+ {
+ // Check the max. dlevel to be dumped
+ if (level > maxLevel)
+ return;
+
+
+ // Create branch
+ for (int i=0; i < level; i++)
+ {
+ if ( i== level-1) std::cout << "| ";
+ else if (masks[i]) std::cout << " ";
+ else std::cout << "| ";
+ }
+ std::cout << std::endl;
+ for (int i=0; i < level-1; i++)
+ {
+ if (masks[i]) std::cout << " ";
+ else std::cout << "| ";
+ }
+
+ // Dump the node name and value
+ std::cout << "|--[" << zkNodeSptr->getKey();
+ if (zkNodeSptr->getData().value != "")
+ std::cout << " => " << zkNodeSptr->getData().value;
+ std::cout << "]" << std::endl;
+
+ // Dump all the children
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ {
+ // Add mask for last child
+ if (i == zkNodeSptr->numChildren()-1)
+ masks.push_back(true);
+ else
+ masks.push_back(false);
+ dumpZkTree_ (zkNodeSptr->getChild (i), maxLevel, level+1, masks);
+ }
+
+ masks.pop_back();
+ return;
+ }
+
+ static ZkTreeNodeSptr traverseBranch_ (const ZkTreeNodeSptr& zkRootSptr,
+ const string& path)
+ {
+ // Check if the tree is loaded into memory
+ if (zkRootSptr == NULL)
+ {
+ string errMsg = "[zktreeutil] null root passed for traversing";
+ std::cout << errMsg << std::endl;
+ throw std::logic_error (errMsg);
+ }
+
+ // Split the path and add intermediate znodes
+ vector< string > nodes;
+ boost::split(nodes, path, boost::is_any_of ("/") );
+
+ // Start traversing the tree
+ ZkTreeNodeSptr currNodeSptr = zkRootSptr;
+ for (unsigned znode_idx = 1; znode_idx < nodes.size(); znode_idx++)
+ {
+ bool found = false;
+ for (unsigned i=0; i < currNodeSptr->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childNodeSptr = currNodeSptr->getChild(i);
+ if (childNodeSptr->getKey() == nodes[znode_idx])
+ {
+ // Found! go to the znode
+ currNodeSptr = childNodeSptr;
+ found = true;
+ break;
+ }
+ }
+ if (!found) // No such znode found; return NULL node-ptr
+ {
+ string errMsg = string("[zktreeutil] unknown znode during traversal: ")
+ + nodes[znode_idx];
+ std::cout << errMsg << std::endl;
+ throw std::logic_error (errMsg);
+ }
+ }
+
+ return currNodeSptr;
+ }
+
+ static ZkTreeNodeSptr createAncestors_ (const string& path)
+ {
+ // Create the root znode
+ ZkTreeNodeSptr zkRootSptr = ZkTreeNodeSptr (new ZkTreeNode ("/"));
+ ZkTreeNodeSptr currNodeSptr = zkRootSptr;
+ // Split the path and add intermediate znodes
+ vector< string > nodes;
+ boost::split(nodes, path, boost::is_any_of ("/") );
+ for (unsigned i=1; i < nodes.size()-1; i++)
+ {
+ ZkTreeNodeSptr childNodeSptr = ZkTreeNodeSptr (new ZkTreeNode (nodes[i]));
+ currNodeSptr->addChild (childNodeSptr);
+ currNodeSptr = childNodeSptr;
+ }
+
+ //Return the root of the branch
+ return zkRootSptr;
+ }
+
+ ZooKeeperAdapterSptr ZkTreeUtil::get_zkHandle (const string& zkHosts)
+ {
+ try
+ {
+ // Create an instance of ZK adapter.
+ ZooKeeperConfig config (zkHosts, 10000);
+ ZooKeeperAdapterSptr zkHandleSptr =
+ ZooKeeperAdapterSptr (new ZooKeeperAdapter (config));
+ return zkHandleSptr;
+ }
+ catch (const ZooKeeperException &e)
+ {
+ std::cerr << "[zktreeutil] zooKeeper exception caught: "
+ << e.what()
+ << std::endl;
+ throw;
+ }
+ catch (std::exception &stde)
+ {
+ std::cerr << "[zktreeutil] standard exception caught: "
+ << stde.what()
+ << std::endl;
+ throw;
+ }
+ catch (...)
+ {
+ std::cerr
+ << "[zktreeutil] unknown exception while connecting to zookeeper"
+ << std::endl;
+ throw;
+ }
+ }
+
+
+ void ZkTreeUtil::loadZkTree (const string& zkHosts,
+ const string& path,
+ bool force)
+ {
+ // Check if already loaded
+ if (loaded_ && !force)
+ {
+ std::cerr << "[zktreeutil] zk-tree already loaded into memory"
+ << std::endl;
+ return;
+ }
+
+ // Connect to ZK server
+ ZooKeeperAdapterSptr zkHandle = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK serverfor reading"
+ << std::endl;
+
+ // Check the existence of the path to znode
+ if (!zkHandle->nodeExists (path))
+ {
+ string errMsg = string("[zktreeutil] path does not exists : ") + path;
+ std::cout << errMsg << std::endl;
+ throw std::logic_error (errMsg);
+ }
+
+ // Load the rooted (sub)tree
+ ZkTreeNodeSptr zkSubrootSptr = loadZkTree_ (zkHandle, path);
+
+ // Create the ancestors before loading the rooted subtree
+ if (path != "/")
+ {
+ zkRootSptr_ = createAncestors_(path);
+ string ppath = path.substr (0, path.rfind('/'));
+ ZkTreeNodeSptr parentSptr = traverseBranch_( zkRootSptr_, ppath);
+ parentSptr->addChild (zkSubrootSptr);
+ }
+ else // Loaded entire zk-tree
+ {
+ zkRootSptr_ = zkSubrootSptr;
+ }
+
+ // Set load flag
+ loaded_ = true;
+ return;
+ }
+
+ void ZkTreeUtil::loadZkTreeXml (const string& zkXmlConfig,
+ bool force)
+ {
+ // Check if already loaded
+ if (loaded_ && !force)
+ {
+ std::cerr << "[zktreeutil] zk-tree already loaded into memory"
+ << std::endl;
+ return;
+ }
+
+ // Parse the file and get the DOM
+ xmlDocPtr docPtr = xmlReadFile(zkXmlConfig.c_str(), NULL, 0);
+ if (docPtr == NULL) {
+ std::cerr << "[zktreeutil] could not parse XML file "
+ << zkXmlConfig
+ << std::endl;
+ exit (-1);
+ }
+ std::cerr << "[zktreeutil] zk-tree XML parsing successful"
+ << std::endl;
+
+ // Get the root element node
+ xmlNodePtr rootPtr = xmlDocGetRootElement(docPtr);
+ // Create the root zk node
+ zkRootSptr_ = ZkTreeNodeSptr (new ZkTreeNode ("/"));
+ // Load the rooted XML tree
+ for (xmlNode* chldNode = rootPtr->children;
+ chldNode;
+ chldNode = chldNode->next)
+ {
+ if (chldNode->type == XML_ELEMENT_NODE)
+ zkRootSptr_->addChild (loadZkTreeXml_ (chldNode));
+ }
+
+ // set oad flag
+ loaded_ = true;
+ // Cleanup stuff
+ xmlFreeDoc(docPtr);
+ xmlCleanupParser();
+ return;
+ }
+
+ void ZkTreeUtil::writeZkTree (const string& zkHosts,
+ const string& path,
+ bool force) const
+ {
+ // Connect to ZK server
+ ZooKeeperAdapterSptr zkHandle = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK server for writing"
+ << std::endl;
+
+ // Go to the rooted subtree
+ ZkTreeNodeSptr zkRootSptr = traverseBranch_ (zkRootSptr_, path);
+
+ // Cleanup before write if forceful write enabled
+ if (force)
+ {
+ if (path != "/") // remove the subtree rooted at the znode
+ {
+ // Delete the subtree rooted at the znode before write
+ if (zkHandle->nodeExists (path))
+ {
+ std::cerr << "[zktreeutil] deleting subtree rooted at "
+ << path
+ << "..."
+ << std::endl;
+ zkHandle->deleteNode (path, true);
+ }
+ }
+ else // remove the rooted znodes
+ {
+ std::cerr << "[zktreeutil] deleting rooted zk-tree"
+ << "..."
+ << std::endl;
+ // Get the root's children
+ vector< string > cnodes = zkHandle->getNodeChildren ("/");
+ for (unsigned i=0; i < cnodes.size(); i++)
+ {
+ if ( cnodes[i] != "/zookeeper") // reserved for zookeeper use
+ zkHandle->deleteNode(cnodes[i], true);
+ }
+ }
+ }
+
+ // Start tree construction
+ writeZkTree_ (zkHandle, zkRootSptr, path);
+ return;
+ }
+
+ void ZkTreeUtil::dumpZkTree (bool xml, int depth) const
+ {
+ if (xml)
+ {
+ // Creates a new document, a node and set it as a root node
+ xmlDocPtr docPtr = xmlNewDoc(BAD_CAST "1.0");
+ xmlNodePtr rootNode = xmlNewNode(NULL, BAD_CAST "root");
+ xmlDocSetRootElement(docPtr, rootNode);
+
+ // Add all the rooted children
+ for (unsigned i=0; i < zkRootSptr_->numChildren(); i++)
+ xmlAddChild (rootNode, dumpZkTreeXml_ (zkRootSptr_->getChild (i)));
+
+ // Dumping document to stdio or file
+ xmlSaveFormatFileEnc("-", docPtr, "UTF-8", 1);
+
+ // Cleanup stuff
+ xmlFreeDoc(docPtr);
+ xmlCleanupParser();
+ return;
+ }
+
+ // Dump text
+ std::cout << "/" << std::endl;
+ vector< bool > masks;
+ for (unsigned i=0; i < zkRootSptr_->numChildren(); i++)
+ {
+ if (i == zkRootSptr_->numChildren()-1)
+ masks.push_back(true);
+ else
+ masks.push_back(false);
+ dumpZkTree_ (zkRootSptr_->getChild (i), depth, 1, masks);
+ }
+
+ return;
+ }
+
+ vector< ZkAction > ZkTreeUtil::diffZkTree (const string& zkHosts,
+ const string& path) const
+ {
+ // Action container
+ vector< ZkAction > actions;
+
+ if (!loaded_)
+ {
+ std::cout << "[zktreeutil] zk-tree not loaded for diff"
+ << std::endl;
+ exit (-1);
+ }
+
+ // Load the rooted subtree from zookeeper
+ ZooKeeperAdapterSptr zkHandle = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK server for reading"
+ << std::endl;
+ ZkTreeNodeSptr zkLiveRootSptr = loadZkTree_ (zkHandle, path);
+
+ // Go to the saved rooted subtree
+ ZkTreeNodeSptr zkLoadedRootSptr =
+ traverseBranch_ (zkRootSptr_, path);
+
+ // Check the root value first
+ if (zkLoadedRootSptr->getData().value
+ != zkLiveRootSptr->getData().value)
+ {
+ actions.push_back (ZkAction (ZkAction::VALUE,
+ path,
+ zkLoadedRootSptr->getData().value,
+ zkLiveRootSptr->getData().value));
+ }
+
+ // Start traversal from root
+ vector< string > ppaths;
+ vector< pair< ZkTreeNodeSptr, ZkTreeNodeSptr > > commonNodes;
+ ppaths.push_back ((path != "/")? path : "");
+ commonNodes.push_back (pair< ZkTreeNodeSptr, ZkTreeNodeSptr >
+ (zkLoadedRootSptr, zkLiveRootSptr));
+
+ for (unsigned j=0; j < commonNodes.size(); j++)
+ {
+ // Get children of loaded tree
+ map< string, ZkTreeNodeSptr > loadedChildren;
+ for (unsigned i=0; i < commonNodes[j].first->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childSptr = commonNodes[j].first->getChild (i);
+ loadedChildren[childSptr->getKey()] = childSptr;
+ }
+ // Get children of live tree
+ map< string, ZkTreeNodeSptr > liveChildren;
+ for (unsigned i=0; i < commonNodes[j].second->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childSptr = commonNodes[j].second->getChild (i);
+ liveChildren[childSptr->getKey()] = childSptr;
+ }
+
+ // Start comparing the children
+ for (map< string, ZkTreeNodeSptr >::const_iterator it =
+ loadedChildren.begin();
+ it != loadedChildren.end();
+ it++)
+ {
+ bool ignoreKey = it->second->getData().ignoreUpdate;
+ string loadedVal = it->second->getData().value;
+ // Path to this node
+ string path = ppaths[j] + string("/") + it->first;
+
+ map< string, ZkTreeNodeSptr >::const_iterator jt =
+ liveChildren.find (it->first);
+ if (jt != liveChildren.end())
+ {
+ // Key is present in live zk-tree
+ string liveVal = jt->second->getData().value;
+ // Check value for the key, if not ignored
+ if (!ignoreKey)
+ {
+ if (loadedVal != liveVal)
+ {
+ // Value differs, set the new value for the key
+ actions.push_back (ZkAction (ZkAction::VALUE,
+ path,
+ loadedVal,
+ liveVal));
+ }
+
+ // Add node to common nodes
+ ppaths.push_back (path);
+ commonNodes.push_back (pair< ZkTreeNodeSptr, ZkTreeNodeSptr >
+ (it->second, jt->second));
+ }
+
+ // Remove the live zk node
+ liveChildren.erase (it->first);
+ }
+ else
+ {
+ // Add the subtree rooted to this node, if not ignored
+ if (!ignoreKey)
+ addTreeZkAction_ (it->second, path, actions);
+ }
+ }
+
+ // Remaining live zk nodes to be deleted
+ for (map< string, ZkTreeNodeSptr >::const_iterator it = liveChildren.begin();
+ it != liveChildren.end(); it++)
+ {
+ string path = ppaths[j] + string("/") + it->first;
+ actions.push_back (ZkAction (ZkAction::DELETE, path));
+ }
+ }
+ // return the diff actions
+ return actions;
+ }
+
+ void ZkTreeUtil::executeZkActions (const string& zkHosts,
+ const vector< ZkAction >& zkActions,
+ int execFlags) const
+ {
+ // Execute the diff zk actions
+ if (zkActions.size())
+ {
+ // Connect to Zookeeper for writing
+ ZooKeeperAdapterSptr zkHandleSptr;
+ if ((execFlags & EXECUTE)
+ || (execFlags & INTERACTIVE))
+ {
+ zkHandleSptr = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK server for writing"
+ << std::endl;
+ }
+
+ for (unsigned i=0; i < zkActions.size(); i++)
+ {
+ if (zkActions[i].action == ZkAction::CREATE)
+ {
+ if (execFlags & PRINT)
+ std::cout << "CREAT- key:" << zkActions[i].key << std::endl;
+ if (execFlags & EXECUTE)
+ {
+ if (execFlags & INTERACTIVE)
+ {
+ string resp;
+ std::cout << "Execute this action?[yes/no]: ";
+ std::getline(std::cin, resp);
+ if (resp != "yes")
+ continue;
+ }
+ zkHandleSptr->createNode(zkActions[i].key.c_str(), "", 0, false);
+ }
+ }
+ else if (zkActions[i].action == ZkAction::DELETE)
+ {
+ if (execFlags & PRINT)
+ std::cout << "DELET- key:" << zkActions[i].key << std::endl;
+ if (execFlags & EXECUTE)
+ {
+ if (execFlags & INTERACTIVE)
+ {
+ string resp;
+ std::cout << "Execute this action?[yes/no]: ";
+ std::getline(std::cin, resp);
+ if (resp != "yes")
+ continue;
+ }
+ zkHandleSptr->deleteNode(zkActions[i].key.c_str(), true);
+ }
+ }
+ else if (zkActions[i].action == ZkAction::VALUE)
+ {
+ if (execFlags & PRINT)
+ {
+ std::cout << "VALUE- key:"
+ << zkActions[i].key
+ << " value:" << zkActions[i].newval;
+ if (zkActions[i].oldval != "")
+ std::cout << " old_value:" << zkActions[i].oldval;
+ std::cout << std::endl;
+ }
+ if (execFlags & EXECUTE)
+ {
+ if (execFlags & INTERACTIVE)
+ {
+ string resp;
+ std::cout << "Execute this action?[yes/no]: ";
+ std::getline(std::cin, resp);
+ if (resp != "yes")
+ continue;
+ }
+ zkHandleSptr->setNodeData (zkActions[i].key, zkActions[i].newval);
+ }
+ }
+ }
+ }
+
+ return;
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.h
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.h b/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.h
new file mode 100644
index 0000000..0a9be03
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtil.h
@@ -0,0 +1,262 @@
+/**
+ * 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.
+ */
+
+#ifndef __ZK_TREE_UTIL_H__
+#define __ZK_TREE_UTIL_H__
+
+#include <libxml/parser.h>
+#include <libxml/tree.h>
+#include "SimpleTree.h"
+#include "ZkAdaptor.h"
+
+namespace zktreeutil
+{
+
+#define ZKTREEUTIL_INF 1000000000
+ /**
+ * \brief A structure containing ZK node data.
+ */
+ struct ZkNodeData
+ {
+ /**
+ * \brief The value string of the ZK node.
+ */
+ string value;
+
+ /**
+ * \brief The flag indicating whether children of the
+ * \brief node shduld be ignored during create/diff/update
+ */
+ bool ignoreUpdate;
+
+ /**
+ * \brief Constructor.
+ *
+ * @param val the value string
+ * @param ignore the flag indicating ignore any update/diff
+ */
+ ZkNodeData (const string& val, bool ignore=false)
+ : value (val), ignoreUpdate (ignore) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param ignore the flag indicating ignore any update/diff
+ */
+ ZkNodeData (bool ignore=false)
+ : ignoreUpdate (ignore) {}
+ };
+
+ /**
+ * \brief The type representing a ZK Treenode
+ */
+ typedef SimpleTreeNode< string, ZkNodeData > ZkTreeNode;
+
+ /**
+ * \brief The type representing a ZK Treenode smart-pointer
+ */
+ typedef boost::shared_ptr< ZkTreeNode > ZkTreeNodeSptr;
+
+ /**
+ * \brief The type representing a ZK Adapter smart-pointer
+ */
+ typedef boost::shared_ptr< ZooKeeperAdapter > ZooKeeperAdapterSptr;
+
+ /**
+ * \brief A structure defining a particular action on ZK node;
+ * \brief the action can be any of -
+ * \brief CREAT- <zknode> : creates <zknode> recussively
+ * \brief DELET- <zknode> : deletes <zknode> recursively
+ * \brief VALUE- <zknode> <value> : sets <value> to <zknode>
+ */
+ struct ZkAction
+ {
+ /**
+ * \brief The action type; any of create/delete/setvalue.
+ */
+ enum ZkActionType
+ {
+ NONE,
+ CREATE,
+ DELETE,
+ VALUE,
+ };
+
+ /**
+ * \brief action of this instance
+ */
+ ZkActionType action;
+
+ /**
+ * \brief ZK node key
+ */
+ string key;
+
+ /**
+ * \brief value to be set, if action is setvalue
+ */
+ string newval;
+
+ /**
+ * \brief existing value of the ZK node key
+ */
+ string oldval;
+
+ /**
+ * \brief Constructor.
+ */
+ ZkAction ()
+ : action (ZkAction::NONE) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param act the action to be taken
+ * @param k the key on which action to be taken
+ */
+ ZkAction (ZkActionType act, const string& k)
+ : action(act),
+ key(k) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param act the action to be taken
+ * @param k the key on which action to be taken
+ * @param v the value of the ZK node key
+ */
+ ZkAction (ZkActionType act, const string& k, const string& v)
+ : action(act),
+ key(k),
+ newval(v) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param act the action to be taken
+ * @param k the key on which action to be taken
+ * @param nv the new value of the ZK node key
+ * @param ov the old value of the ZK node key
+ */
+ ZkAction (ZkActionType act, const string& k, const string& nv, const string& ov)
+ : action (act),
+ key(k),
+ newval(nv),
+ oldval(ov) {}
+ };
+
+ /**
+ * \brief The ZK tree utility class; supports loading ZK tree from ZK server OR
+ * \brief from saved XML file, saving ZK tree into XML file, dumping the ZK tree
+ * \brief on standard output, creting a diff between saved ZK tree and live ZK
+ * \brief tree and incremental update of the live ZK tree.
+ */
+ class ZkTreeUtil
+ {
+ public:
+ /**
+ * \brief Execution flag on ZkAction
+ */
+ enum ZkActionExecuteFlag
+ {
+ NONE = 0,
+ PRINT = 1,
+ EXECUTE = 2,
+ INTERACTIVE = 5,
+ };
+
+ public:
+ /**
+ * \brief Connects to zookeeper and returns a valid ZK handle
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param a valid ZK handle
+ */
+ static ZooKeeperAdapterSptr get_zkHandle (const string& zkHosts);
+
+
+ public:
+ /**
+ * \brief Constructor.
+ */
+ ZkTreeUtil () : loaded_(false) {}
+
+ /**
+ * \brief loads the ZK tree from ZK server into memory
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param path path to the subtree to be loaded into memory
+ * @param force forces reloading in case tree already loaded into memory
+ */
+ void loadZkTree (const string& zkHosts, const string& path="/", bool force=false);
+
+ /**
+ * \brief loads the ZK tree from XML file into memory
+ *
+ * @param zkXmlConfig ZK tree XML file
+ * @param force forces reloading in case tree already loaded into memory
+ */
+ void loadZkTreeXml (const string& zkXmlConfig, bool force=false);
+
+ /**
+ * \brief writes the in-memory ZK tree on to ZK server
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param path path to the subtree to be written to ZK tree
+ * @param force forces cleanup of the ZK tree on the ZK server before writing
+ */
+ void writeZkTree (const string& zkHosts, const string& path="/", bool force=false) const;
+
+ /**
+ * \brief dupms the in-memory ZK tree on the standard output device;
+ *
+ * @param xml flag indicates whether tree should be dumped in XML format
+ * @param depth the depth of the tree to be dumped for non-xml dump
+ */
+ void dumpZkTree (bool xml=false, int depth=ZKTREEUTIL_INF) const;
+
+ /**
+ * \brief returns a list of actions after taking a diff of in-memory
+ * \brief ZK tree and live ZK tree.
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param path path to the subtree in consideration while taking diff with ZK tree
+ * @return a list of ZKAction instances to be performed on live ZK tree
+ */
+ vector< ZkAction > diffZkTree (const string& zkHosts, const string& path="/") const;
+
+ /**
+ * \brief performs create/delete/setvalue by executing a set of
+ * ZkActions on a live ZK tree.
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param zkActions set of ZkActions
+ * @param execFlags flags indicating print/execute/interactive etc
+ */
+ void executeZkActions (const string& zkHosts,
+ const vector< ZkAction >& zkActions,
+ int execFlags) const;
+
+ private:
+
+ ZkTreeNodeSptr zkRootSptr_; // ZK tree root node
+ bool loaded_; // Falg indicating whether ZK tree loaded into memory
+ };
+}
+
+#endif // __ZK_TREE_UTIL_H__
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtilMain.cc
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtilMain.cc b/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtilMain.cc
new file mode 100644
index 0000000..8afebf6
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zktreeutil/src/ZkTreeUtilMain.cc
@@ -0,0 +1,247 @@
+/**
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#include <config.h>
+#endif
+
+#include <unistd.h>
+#ifndef _GNU_SOURCE
+#define _GNU_SOURCE
+#endif
+#include <getopt.h>
+#include <iostream>
+#include "ZkTreeUtil.h"
+
+using namespace zktreeutil;
+
+// The set of "long" options accepted by this program.
+static struct option long_options[] = {
+ {"help", no_argument, 0, 'h'},
+ {"import", no_argument, 0, 'I'},
+ {"export", no_argument, 0, 'E'},
+ {"update", no_argument, 0, 'U'},
+ {"diff", no_argument, 0, 'F'},
+ {"dump", no_argument, 0, 'D'},
+ {"force", no_argument, 0, 'f'},
+ {"xmlfile", required_argument, 0, 'x'},
+ {"path", required_argument, 0, 'p'},
+ {"depth", required_argument, 0, 'd'},
+ {"zookeeper", required_argument, 0, 'z'},
+ {0, 0, 0, 0}
+};
+static char *short_options = "IEUFDfx:p:d:hz:";
+
+static void usage(int argc, char *argv[])
+{
+ std::cout << "ZK-tree utility for managing ZK-tree with XML import/export," << std::endl;
+ std::cout << "viewing diff between live and saved ZK-tree and performing" << std::endl;
+ std::cout << "incremental update of the same." << std::endl;
+ std::cout << "Usage: " << argv[0] << " [args-and-values]+" << std::endl;
+ std::cout
+ << "\t--import or -I: "
+ << std::endl
+ << "\t Imports the zookeeper tree from XML file. Must be specified with"
+ << std::endl
+ << "\t --zookeeper AND --xmlfile options. Optionally takes --path for"
+ << std::endl
+ << "\t importing subtree"
+ << std::endl;
+ std::cout
+ << "\t--export or -E: "
+ << std::endl
+ << "\t Exports the zookeeper tree to XML file. Must be specified with"
+ << std::endl
+ << "\t --zookeeper option. Optionally takes --path for exporting subtree"
+ << std::endl;
+ std::cout
+ << "\t--update or -U: "
+ << std::endl
+ << "\t Updates zookeeper tree with changes from XML file. Update operation"
+ << std::endl
+ << "\t is interactive unless specified with --force option. Must be speci-"
+ << std::endl
+ << "\t fied with --zookeeper AND --xmlfile options. Optionally takes --path"
+ << std::endl
+ << "\t for updating subtree."
+ << std::endl;
+ std::cout
+ << "\t--diff or -F: "
+ << std::endl
+ << "\t Creates a list of diff actions on ZK tree based on XML data. Must"
+ << std::endl
+ << "\t be specified with --zookeeper OR --xmlfile options. Optionally takes"
+ << std::endl
+ << "\t --path for subtree diff"
+ << std::endl;
+ std::cout
+ << "\t--dump or -D: "
+ << std::endl
+ << "\t Dumps the entire ZK (sub)tree to standard output. Must be specified"
+ << std::endl
+ << "\t with --zookeeper OR --xmlfile options. Optionally takes --path and"
+ << std::endl
+ << "\t --depth for dumping subtree."
+ << std::endl;
+ std::cout
+ << "\t--xmlfile=<filename> or -x <filename>: "
+ << std::endl
+ << "\t Zookeeper tree-data XML file."
+ << std::endl;
+ std::cout
+ << "\t--path=<znodepath> or -p <znodepath>: "
+ << std::endl
+ << "\t Path to the zookeeper subtree rootnode."
+ << std::endl;
+ std::cout
+ << "\t--depth=<tree-depth> or -d <tree-depth>: "
+ << std::endl
+ << "\t Depth of the ZK tree to be dumped (ignored for XML dump)."
+ << std::endl;
+ std::cout
+ << "\t--force or -f: Forces cleanup before import; also used for forceful"
+ << std::endl
+ << "\t update. Optionally be specified with --import and --update."
+ << std::endl;
+ std::cout
+ << "\t--help or -h: "
+ << std::endl
+ << "\t prints this message"
+ << std::endl;
+ std::cout
+ << "\t--zookeeper=<zkhosts> or -z <zkhosts>: "
+ << std::endl
+ << "\t specifies information to connect to zookeeper."
+ << std::endl;
+}
+
+int main(int argc, char **argv)
+{
+ if (argc == 1) {
+ usage(argc, argv);
+ exit(0);
+ }
+
+ // Parse the arguments.
+ int op = 0;
+ bool force = false;
+ string zkHosts;
+ string xmlFile;
+ string path = "/";
+ int depth = 0;
+ while (1)
+ {
+ int c = getopt_long(argc, argv, short_options, long_options, 0);
+ if (c == -1)
+ break;
+
+ switch (c) {
+ case 'I': op = c;
+ break;
+ case 'E': op = c;
+ break;
+ case 'U': op = c;
+ break;
+ case 'F': op = c;
+ break;
+ case 'D': op = c;
+ break;
+ case 'f': force = true;
+ break;
+ case 'x': xmlFile = optarg;
+ break;
+ case 'p': path = optarg;
+ break;
+ case 'd': depth = atoi (optarg);
+ break;
+ case 'z': zkHosts = optarg;
+ break;
+ case 'h': usage (argc, argv);
+ exit(0);
+ }
+ }
+
+ ZkTreeUtil zkTreeUtil;
+ switch (op)
+ {
+ case 'I': {
+ if (zkHosts == "" || xmlFile == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ zkTreeUtil.writeZkTree (zkHosts, path, force);
+ std::cout << "[zktreeutil] import successful!" << std::endl;
+ break;
+ }
+ case 'E': {
+ if (zkHosts == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTree (zkHosts, path);
+ zkTreeUtil.dumpZkTree (true);
+ break;
+ }
+ case 'U': {
+ if (zkHosts == "" || xmlFile == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ vector< ZkAction > zkActions = zkTreeUtil.diffZkTree (zkHosts, path);
+ int flags = ZkTreeUtil::EXECUTE;
+ if (!force) flags |= ZkTreeUtil::INTERACTIVE;
+ zkTreeUtil.executeZkActions (zkHosts, zkActions, flags);
+ std::cout << "[zktreeutil] update successful!" << std::endl;
+ break;
+ }
+ case 'F': {
+ if (zkHosts == "" || xmlFile == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ vector< ZkAction > zkActions = zkTreeUtil.diffZkTree (zkHosts, path);
+ zkTreeUtil.executeZkActions (zkHosts, zkActions, ZkTreeUtil::PRINT);
+ break;
+ }
+ case 'D': {
+ if (zkHosts != "")
+ zkTreeUtil.loadZkTree (zkHosts, path);
+ else if (xmlFile != "")
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ else
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ // Dump the ZK tree
+ if (depth) zkTreeUtil.dumpZkTree (false, depth);
+ else zkTreeUtil.dumpZkTree (false);
+ break;
+ }
+ }
+
+ exit(0);
+}
+
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zktreeutil/tests/zk_sample.xml
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zktreeutil/tests/zk_sample.xml b/zookeeper-contrib/zookeeper-contrib-zktreeutil/tests/zk_sample.xml
new file mode 100644
index 0000000..6e97daa
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zktreeutil/tests/zk_sample.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
+<root>
+ <zknode name="myapp">
+ <zknode name="version-1.0">
+ <zknode name="clientConfig">
+ <zknode name="testClient" value="cluster.id=local;server.host=localhost;server.port=4080"/>
+ </zknode>
+ <zknode name="configuration" value="v4.0">
+ <zknode name="cacheControl" value="on"/>
+ <zknode name="healthCheck" value="on"/>
+ </zknode>
+ <zknode name="distributions">
+ <zknode name="http">
+ <zknode name="goldenShards" value="0,4294967296,server,localhost:8085;"/>
+ <zknode name="versionedShards" value="33;0,4294967296,server,localhost:8086;"/>
+ <zknode name="shards" value="0,4294967296,server,localhost:8086;"/>
+ </zknode>
+ </zknode>
+ <zknode name="tmp" ignore="yes">
+ <zknode name="alerts" value="test"/>
+ <zknode name="locks"/>
+ <zknode name="transactions"/>
+ </zknode>
+ </zknode>
+ </zknode>
+ <zknode name="zookeeper" ignore="true"/>
+</root>
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zooinspector/NOTICE.txt
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/NOTICE.txt b/zookeeper-contrib/zookeeper-contrib-zooinspector/NOTICE.txt
new file mode 100644
index 0000000..5420ef0
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/NOTICE.txt
@@ -0,0 +1,9 @@
+src/java/com/nitido/utils/toaster/Toaster.java:
+This java file is copyright by Daniele Piras ("danielepiras80", no email known) released under the Apache Software License 2.0
+It has been downloaded in december 2009 from the CVS web interface of the sourceforge project http://sourceforge.net/projects/jtoaster/ . The web interface to CVS is not available anymore on sourceforge.
+
+The icons in src/main/resources/icons are taken from the Tango project
+downloaded from http://tango.freedesktop.org/releases on 2011-09-06.
+The Tango project is public domain.
+
+Distribution packagers should not include the icons in the package but rather depend on tango-icon-theme (Debian package name). ZooInspector will then try to get the icons from /usr/share/icons/Tango rather then from its jar file.
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zooinspector/README.txt
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/README.txt b/zookeeper-contrib/zookeeper-contrib-zooinspector/README.txt
new file mode 100644
index 0000000..3c2a58f
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/README.txt
@@ -0,0 +1,94 @@
+==========================================
+ZooInspector - Browser and Editor for ZooKeeper Instances
+Author: Colin Goodheart-Smithe
+Date: February 2010
+==========================================
+
+ZooInspector is a Java Swing based application for browsing and editing ZooKeeper instances.
+
+Contents
+--------
+ - Features
+ - Pre-requisites
+ - Build Instructions
+ - Using ZooInspector
+ - Creating and Using Plugins
+
+Features
+--------
+ Below is a list of features in the current release of ZooInspector.
+ - Load connection settings from a zookeeper properties file
+ - Plugable DataEncryptionManagers to specify how data should be encrypted and decrypted in the Zookeeper instance
+ - Browseable tree view of the ZooKeeper instance
+ - View the data in a node
+ - View the ACL's currently applied to a node
+ - View the metadata for a node (Version, Number of Children, Last modified Tiem, etc.)
+ - Plugable NodeViewers interface
+ - Ability to save/load and set default Node Viewers
+
+Pre-requisites
+--------------
+ - The main zookeeper build script must have been run before building this module
+
+Build Instructions
+------------------
+ 1. Open a command line.
+ 2. cd into this directory
+ 3. Run command: ant
+ 4. ZooInspector will be built to ../../../build/contrib/ZooInspector
+ 5. Copy zookeeper-3.x.x.jar into the lib sub-directory (if you are using zookeeper-3.3.0.jar it will have been
+ copied to this directory during the build
+ 6. By default the zookeeper.cmd and zookeeper.sh files expect zookeeper-3.3.0.jar. If you are using another version
+ you will need to change these files to point to the zookeeper-3.x.x.jar you copied to the lib directory
+ 7. To run ZooInspector run zooInspector.cmd (on Windows) or zooInspector.sh (on Linux). If you are using
+ zookeeper-3.3.0.jar and do not require any classpath changes you can run the zookeeper-dev-ZooInspector.jar
+ directly
+
+Using ZooInspector
+------------------
+ To start ZooInspector run zooInspector.cmd (on Windows) or zooInspector.sh (on Linux). If you are using
+ zookeeper-3.3.0.jar and do not require any classpath changes you can run the zookeeper-dev-ZooInspector.jar
+ directly.
+
+ Click the play button on the toolbar to bring up the connection dialog. From here you can enter connection
+ information for your zookeeper instance. You can also load the connection properties from a file. This file can
+ have the format as a normal zookeeper properties file (i.e. hosts and timeout key-value pairs) and van optional have
+ an encryptionManager key-value pair to specify the DataEncryptionManager to use for this connection
+ (DataEncryptionManagers are explained in further detail in the 'Creating and Using Plugins' section below). You can
+ also set the entered information as the defaults so that when you first start ZooInspector these settings are
+ automatically loaded into this dialog. Pressing the OK button with connect to your ZooKeeper instance and show the
+ current node tree on the left of the main panel.
+
+ Clicking a node in the node tree will load the data for that node into the node viewers. Three node viewers are
+ currently distributed with ZooInspector:
+ 1. Node Data - This enables you to see the data current stored on that node. This data can be modified and
+ saved. The data is decrypted and encrypted using the DataEncryptionManager specified on the connection
+ dialog.
+ 2. Node Metadata - This enables you to see the metadata associiated with this node. This is Essentially the data
+ obtained from the Stat object for this node.
+ 3. Node ACLs - This allows you to see the ACLs currently applied to this node. Currently there is no ability
+ to change the ACLs on a node, but it is a feature I would like to add.
+ Other custom Node Viewers can be added, this is explained in the 'Creating and Using Plugins' section below.
+
+
+Creating and Using Plugins
+--------------------------
+ There are two types of plugin which can be used with ZooInspector:
+ 1. DataEncryptionManager - This specifies how data should be encrypted and decrypted when working with a
+ zookeeper instance.
+ 2. ZooInspectorNodeViewer - This is a GUI panel which provides a view of visualisation on a node.
+ More information on these interfaces can be found in the javadocs for this module.
+
+ To use a plugin in ZooInspector, build the plugin to a jar and copy the jar to the lib sub-directory. Edit the
+ zooInspector.cmd and/or zooInspector.sh files to include your new jar on the classpath and run ZooInspector.
+
+ For DataEncryptionManagers, click the play button to open the connection dialog and enter the full class name of
+ your DataEncryptionManager in the 'Data Encryption Manager' field. You can make this Data Encryption Manager the
+ default by clicking 'Set As Default'. Click the 'OK' button to instantiate and use your plugin.
+
+ For ZooInspectorNodeViewers, Click the 'Change Node Viewers' button on the toolbar (looks like a tree with a pencil)
+ and enter the full classname for your Node Viewer in the field left of the 'Add' button, then click the 'Add'
+ button. The Node Viewer will be instantiated and should appear in the list. You can change the order of the Node
+ viewers by clicking the up and dpwn buttons and delete a Node Viewer by clicking the delete button. You can save
+ to configuration to a file or set it as the default if necessary. Then click the 'OK' button and your Node Viewer
+ should appear in the tabs on the right of the main panel.
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zooinspector/TODO
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/TODO b/zookeeper-contrib/zookeeper-contrib-zooinspector/TODO
new file mode 100644
index 0000000..404d5c9
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/TODO
@@ -0,0 +1,19 @@
+- replace JToaster with standard notifications, see:
+ http://www.galago-project.org/specs/notification/
+ http://stackoverflow.com/questions/857154/freedesktop-org-notifications-in-java
+ DBus and Java:
+ http://bolta-gecko.blogspot.com/2009/06/using-d-bus-in-java.html
+ http://dbus.freedesktop.org/doc/dbus-java/ (packaged in Debian)
+
+- properly respect
+ http://standards.freedesktop.org/desktop-entry-spec/desktop-entry-spec-latest.html
+ http://standards.freedesktop.org/basedir-spec/basedir-spec-latest.html
+ http://standards.freedesktop.org/icon-theme-spec/icon-theme-spec-latest.html
+
+- Rename classes to avoid redundand "ZooInspector" prefix.
+
+- Ant build file has hard coded log4j dependency. (ZK will move to maven anyways...)
+
+- make directory for config files configurable via commandline parameter
+
+- Clean up the code! :-)
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zooinspector/build.xml
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/build.xml b/zookeeper-contrib/zookeeper-contrib-zooinspector/build.xml
new file mode 100644
index 0000000..68ea9f7
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/build.xml
@@ -0,0 +1,148 @@
+<!--
+ 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.
+-->
+
+<project name="ZooInspector" default="jar">
+ <import file="../build-contrib.xml" />
+
+
+ <target name="setjarname">
+ <property name="jarname" value="${build.dir}/zookeeper-${version}-${name}.jar" />
+ </target>
+
+ <target name="init" depends="checkMainCompiled, zookeeperbuildcontrib.init">
+ <mkdir dir="${build.dir}/licences" />
+ <copy todir="${build.dir}/licences">
+ <fileset dir="${basedir}/licences" />
+ </copy>
+ <mkdir dir="${build.dir}/icons" />
+ <copy todir="${build.dir}/icons">
+ <fileset dir="${basedir}/src/main/resources/icons" />
+ </copy>
+ <mkdir dir="${build.dir}/config" />
+ <copy todir="${build.dir}/config">
+ <fileset dir="${basedir}/src/main/resources">
+ <include name="defaultConnectionSettings.cfg" />
+ <include name="defaultNodeViewers.cfg" />
+ </fileset>
+ </copy>
+ <copy todir="${build.dir}/lib">
+ <fileset file="${basedir}/src/main/resources/log4j.properties" />
+ </copy>
+ <copy todir="${build.dir}/lib">
+ <fileset file="../../build/zookeeper-${version}.jar" />
+ </copy>
+ <copy todir="${build.dir}">
+ <fileset dir="${basedir}" includes="*.*" excludes="build.xml,ivy.xml" />
+ </copy>
+ </target>
+
+ <!-- Override jar target to specify main class -->
+ <target name="jar" depends="setjarname, compile">
+ <echo message="contrib: ${name}" />
+
+ <jar jarfile="${jarname}">
+ <manifest>
+ <attribute name="Main-Class" value="org.apache.zookeeper.inspector.ZooInspector" />
+ <attribute name="Built-By" value="${user.name}" />
+ <attribute name="Built-At" value="${build.time}" />
+ <attribute name="Built-On" value="${host.name}" />
+ <attribute name="Implementation-Title" value="org.apache.zookeeper" />
+ <attribute name="Implementation-Version" value="${revision}" />
+ <attribute name="Implementation-Vendor" value="The Apache Software Foundation" />
+ </manifest>
+ <fileset file="${zk.root}/LICENSE.txt" />
+ <fileset dir="${build.classes}" />
+ <fileset dir="${basedir}/src/main/resources">
+ <exclude name="*.cfg" />
+ <exclude name="*.properties" />
+ </fileset>
+ <fileset dir="${basedir}/src/main/java" excludes="**/*.jar, **/*.java"/>
+ </jar>
+ </target>
+
+ <target name="compile" depends="ivy-retrieve,zookeeperbuildcontrib.compile" />
+
+ <target name="test" depends="checkMainTestCompiled,compile-test,test-init,test-category,junit.run" />
+
+ <target name="compile-test" depends="ivy-retrieve-test,compile">
+ <property name="target.jdk" value="${ant.java.version}" />
+ <property name="src.test.local" location="${basedir}/test" />
+ <mkdir dir="${build.test}" />
+ <javac srcdir="${src.test.local}" destdir="${build.test}" target="${target.jdk}" debug="on" encoding="${build.encoding}">
+ <classpath refid="classpath" />
+ <classpath>
+ <pathelement location="${zk.root}/build/test/classes" />
+ </classpath>
+ </javac>
+ </target>
+
+ <target name="test-init" depends="jar,compile-test">
+ <delete dir="${test.log.dir}" />
+ <delete dir="${test.tmp.dir}" />
+ <delete dir="${test.data.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ <mkdir dir="${test.tmp.dir}" />
+ <mkdir dir="${test.data.dir}" />
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value="" />
+ </target>
+
+ <target name="junit.run">
+ <echo message="${test.src.dir}" />
+ <junit showoutput="${test.output}" printsummary="${test.junit.printsummary}" haltonfailure="${test.junit.haltonfailure}" fork="yes" forkmode="${test.junit.fork.mode}" maxmemory="${test.junit.maxmem}" dir="${basedir}" timeout="${test.timeout}" errorProperty="tests.failed" failureProperty="tests.failed">
+ <sysproperty key="build.test.dir" value="${test.tmp.dir}" />
+ <sysproperty key="test.data.dir" value="${test.data.dir}" />
+ <sysproperty key="log4j.configuration" value="file:${basedir}/conf/log4j.properties" />
+ <classpath refid="classpath" />
+ <classpath>
+ <pathelement path="${build.test}" />
+ <pathelement location="${zk.root}/build/test/classes" />
+ </classpath>
+ <formatter type="${test.junit.output.format}" />
+ <batchtest todir="${test.log.dir}" unless="testcase">
+ <fileset dir="${test.src.dir}" includes="**/*${test.category}Test.java" />
+ </batchtest>
+ <batchtest todir="${test.log.dir}" if="testcase">
+ <fileset dir="${test.src.dir}" includes="**/${testcase}.java" />
+ </batchtest>
+ </junit>
+ <fail if="tests.failed">Tests failed!</fail>
+ </target>
+
+ <target name="package" depends="jar, zookeeperbuildcontrib.package" unless="skip.contrib">
+
+ <copy file="${basedir}/build.xml" todir="${dist.dir}/contrib/${name}" />
+
+ <mkdir dir="${dist.dir}/contrib/${name}/src" />
+ <copy todir="${dist.dir}/contrib/${name}/src">
+ <fileset dir="${basedir}/src" />
+ </copy>
+ <mkdir dir="${dist.dir}/contrib/${name}/licences" />
+ <copy todir="${dist.dir}/contrib/${name}/licences">
+ <fileset dir="${basedir}/licences" />
+ </copy>
+ <mkdir dir="${dist.dir}/contrib/${name}/config" />
+ <copy todir="${dist.dir}/contrib/${name}/config">
+ <fileset dir="${basedir}/src/main/resources">
+ <include name="defaultConnectionSettings.cfg" />
+ <include name="defaultNodeViewers.cfg" />
+ </fileset>
+ </copy>
+ </target>
+</project>
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zooinspector/ivy.xml
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/ivy.xml b/zookeeper-contrib/zookeeper-contrib-zooinspector/ivy.xml
new file mode 100644
index 0000000..d841d18
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/ivy.xml
@@ -0,0 +1,50 @@
+<!--
+ 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.
+-->
+
+<ivy-module version="2.0"
+ xmlns:e="http://ant.apache.org/ivy/extra">
+
+ <info organisation="org.apache.zookeeper"
+ module="${name}" revision="${version}">
+ <license name="Apache 2.0"/>
+ <ivyauthor name="Apache Hadoop" url="http://hadoop.apache.org"/>
+ <description>ZooInspector</description>
+ </info>
+
+ <configurations defaultconfmapping="default">
+ <conf name="default"/>
+ <conf name="test"/>
+ <conf name="releaseaudit" visibility="private" description="Artifacts required for releaseaudit target"/>
+ </configurations>
+
+ <dependencies>
+ <dependency org="com.google.guava" name="guava" rev="18.0" />
+
+ <dependency org="org.slf4j" name="slf4j-api" rev="1.7.5"/>
+ <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.7.5" transitive="false"/>
+
+ <dependency org="log4j" name="log4j" rev="1.2.17" transitive="false"/>
+ <dependency org="junit" name="junit" rev="4.12" conf="test->default"/>
+ <dependency org="org.apache.rat" name="apache-rat-tasks"
+ rev="0.6" conf="releaseaudit->default"/>
+ <dependency org="commons-lang" name="commons-lang"
+ rev="2.4" conf="releaseaudit->default"/>
+ <dependency org="commons-collections" name="commons-collections"
+ rev="3.2.2" conf="releaseaudit->default"/>
+ </dependencies>
+
+</ivy-module>
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zooinspector/licences/Apache Software Licence v2.0.txt
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/licences/Apache Software Licence v2.0.txt b/zookeeper-contrib/zookeeper-contrib-zooinspector/licences/Apache Software Licence v2.0.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/licences/Apache Software Licence v2.0.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/b0df8fe1/zookeeper-contrib/zookeeper-contrib-zooinspector/src/main/java/com/nitido/utils/toaster/Toaster.java
----------------------------------------------------------------------
diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/src/main/java/com/nitido/utils/toaster/Toaster.java b/zookeeper-contrib/zookeeper-contrib-zooinspector/src/main/java/com/nitido/utils/toaster/Toaster.java
new file mode 100644
index 0000000..cc123ba
--- /dev/null
+++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/src/main/java/com/nitido/utils/toaster/Toaster.java
@@ -0,0 +1,533 @@
+/**
+ * This java file is copyright by Daniele Piras ("danielepiras80", no email known) released under the
+ * Apache Software License 2.0. It has been downloaded in december 2009 from the CVS web interface
+ * of the sourceforge project http://sourceforge.net/projects/jtoaster/ . The web interface to CVS
+ * is not available anymore on sourceforge.
+ *
+ */
+
+/**
+ * Java Toaster is a java utility class for your swing applications
+ * that show an animate box coming from the bottom of your screen
+ * with a notification message and/or an associated image
+ * (like msn online/offline notifications).
+ *
+ * Toaster panel in windows system follow the taskbar; So if
+ * the taskbar is into the bottom the panel coming from the bottom
+ * and if the taskbar is on the top then the panel coming from the top.
+ *
+ * This is a simple example of utilization:
+ *
+ * import com.nitido.utils.toaster.*;
+ * import javax.swing.*;
+ *
+ * public class ToasterTest
+ * {
+ *
+ * public static void main(String[] args)
+ * {
+ * // Initialize toaster manager...
+ * Toaster toasterManager = new Toaster();
+ *
+ * // Show a simple toaster
+ * toasterManager.showToaster( new ImageIcon( "mylogo.gif" ), "A simple toaster with an image" );
+ * }
+ * }
+ */
+package com.nitido.utils.toaster;
+
+import java.awt.*;
+
+import javax.swing.*;
+import javax.swing.border.*;
+
+/**
+ * Class to show tosters in multiplatform
+ *
+ * @author daniele piras
+ *
+ */
+public class Toaster
+{
+ // Width of the toster
+ private int toasterWidth = 300;
+
+ // Height of the toster
+ private int toasterHeight = 80;
+
+ // Step for the toaster
+ private int step = 20;
+
+ // Step time
+ private int stepTime = 20;
+
+ // Show time
+ private int displayTime = 3000;
+
+ // Current number of toaster...
+ private int currentNumberOfToaster = 0;
+
+ // Last opened toaster
+ private int maxToaster = 0;
+
+ // Max number of toasters for the sceen
+ private int maxToasterInSceen;
+
+ // Font used to display message
+ private Font font;
+
+ // Color for border
+ private Color borderColor;
+
+ // Color for toaster
+ private Color toasterColor;
+
+ // Set message color
+ private Color messageColor;
+
+ // Set the margin
+ int margin;
+
+ // Flag that indicate if use alwaysOnTop or not.
+ // method always on top start only SINCE JDK 5 !
+ boolean useAlwaysOnTop = true;
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Constructor to initialized toaster component...
+ *
+ * @author daniele piras
+ *
+ */
+ public Toaster()
+ {
+ // Set default font...
+ font = new Font("Arial", Font.BOLD, 12);
+ // Border color
+ borderColor = new Color(245, 153, 15);
+ toasterColor = Color.WHITE;
+ messageColor = Color.BLACK;
+ useAlwaysOnTop = true;
+ // Verify AlwaysOnTop Flag...
+ try
+ {
+ JWindow.class.getMethod( "setAlwaysOnTop", new Class[] { Boolean.class } );
+ }
+ catch( Exception e )
+ {
+ useAlwaysOnTop = false;
+ }
+
+ }
+
+ /**
+ * Class that rappresent a single toaster
+ *
+ * @author daniele piras
+ *
+ */
+ class SingleToaster extends javax.swing.JWindow
+ {
+ private static final long serialVersionUID = 1L;
+
+ // Label to store Icon
+ private JLabel iconLabel = new JLabel();
+
+ // Text area for the message
+ private JTextArea message = new JTextArea();
+
+
+
+
+ /***
+ * Simple costructor that initialized components...
+ */
+ public SingleToaster()
+ {
+ initComponents();
+ }
+
+ /***
+ * Function to initialized components
+ */
+ private void initComponents()
+ {
+
+ setSize(toasterWidth, toasterHeight);
+ message.setFont( getToasterMessageFont() );
+ JPanel externalPanel = new JPanel(new BorderLayout(1, 1));
+ externalPanel.setBackground( getBorderColor() );
+ JPanel innerPanel = new JPanel(new BorderLayout( getMargin(), getMargin() ));
+ innerPanel.setBackground( getToasterColor() );
+ message.setBackground( getToasterColor() );
+ message.setMargin( new Insets( 2,2,2,2 ) );
+ message.setLineWrap( true );
+ message.setWrapStyleWord( true );
+
+ EtchedBorder etchedBorder = (EtchedBorder) BorderFactory
+ .createEtchedBorder();
+ externalPanel.setBorder(etchedBorder);
+
+ externalPanel.add(innerPanel);
+ message.setForeground( getMessageColor() );
+ innerPanel.add(iconLabel, BorderLayout.WEST);
+ innerPanel.add(message, BorderLayout.CENTER);
+ getContentPane().add(externalPanel);
+ }
+
+
+ /***
+ * Start toaster animation...
+ */
+ public void animate()
+ {
+ ( new Animation( this ) ).start();
+ }
+
+ }
+
+ /***
+ * Class that manage the animation
+ */
+ class Animation extends Thread
+ {
+ SingleToaster toaster;
+
+ public Animation( SingleToaster toaster )
+ {
+ this.toaster = toaster;
+ }
+
+
+ /**
+ * Animate vertically the toaster. The toaster could be moved from bottom
+ * to upper or to upper to bottom
+ * @param posx
+ * @param fromy
+ * @param toy
+ * @throws InterruptedException
+ */
+ protected void animateVertically( int posx, int fromY, int toY ) throws InterruptedException
+ {
+
+ toaster.setLocation( posx, fromY );
+ if ( toY < fromY )
+ {
+ for (int i = fromY; i > toY; i -= step)
+ {
+ toaster.setLocation(posx, i);
+ Thread.sleep(stepTime);
+ }
+ }
+ else
+ {
+ for (int i = fromY; i < toY; i += step)
+ {
+ toaster.setLocation(posx, i);
+ Thread.sleep(stepTime);
+ }
+ }
+ toaster.setLocation( posx, toY );
+ }
+
+ public void run()
+ {
+ try
+ {
+ boolean animateFromBottom = true;
+ GraphicsEnvironment ge = GraphicsEnvironment
+ .getLocalGraphicsEnvironment();
+ Rectangle screenRect = ge.getMaximumWindowBounds();
+
+ int screenHeight = (int) screenRect.height;
+
+ int startYPosition;
+ int stopYPosition;
+
+ if ( screenRect.y > 0 )
+ {
+ animateFromBottom = false; // Animate from top!
+ }
+
+ maxToasterInSceen = screenHeight / toasterHeight;
+
+
+ int posx = (int) screenRect.width - toasterWidth - 1;
+
+ toaster.setLocation(posx, screenHeight);
+ toaster.setVisible(true);
+ if ( useAlwaysOnTop )
+ {
+ toaster.setAlwaysOnTop(true);
+ }
+
+ if ( animateFromBottom )
+ {
+ startYPosition = screenHeight;
+ stopYPosition = startYPosition - toasterHeight - 1;
+ if ( currentNumberOfToaster > 0 )
+ {
+ stopYPosition = stopYPosition - ( maxToaster % maxToasterInSceen * toasterHeight );
+ }
+ else
+ {
+ maxToaster = 0;
+ }
+ }
+ else
+ {
+ startYPosition = screenRect.y - toasterHeight;
+ stopYPosition = screenRect.y;
+
+ if ( currentNumberOfToaster > 0 )
+ {
+ stopYPosition = stopYPosition + ( maxToaster % maxToasterInSceen * toasterHeight );
+ }
+ else
+ {
+ maxToaster = 0;
+ }
+ }
+
+ currentNumberOfToaster++;
+ maxToaster++;
+
+
+ animateVertically( posx, startYPosition, stopYPosition );
+ Thread.sleep(displayTime);
+ animateVertically( posx, stopYPosition, startYPosition );
+
+ currentNumberOfToaster--;
+ toaster.setVisible(false);
+ toaster.dispose();
+ } catch (Exception e)
+ {
+ e.printStackTrace();
+ }
+ }
+ }
+
+
+
+ /**
+ * Show a toaster with the specified message and the associated icon.
+ */
+ public void showToaster(Icon icon, String msg)
+ {
+ SingleToaster singleToaster = new SingleToaster();
+ if ( icon != null )
+ {
+ singleToaster.iconLabel.setIcon( icon );
+ }
+ singleToaster.message.setText( msg );
+ singleToaster.animate();
+ }
+
+ /**
+ * Show a toaster with the specified message.
+ */
+ public void showToaster( String msg )
+ {
+ showToaster( null, msg );
+ }
+
+ /**
+ * @return Returns the font
+ */
+ public Font getToasterMessageFont()
+ {
+ // TODO Auto-generated method stub
+ return font;
+ }
+
+ /**
+ * Set the font for the message
+ */
+ public void setToasterMessageFont( Font f)
+ {
+ font = f;
+ }
+
+
+ /**
+ * @return Returns the borderColor.
+ */
+ public Color getBorderColor()
+ {
+ return borderColor;
+ }
+
+
+
+ /**
+ * @param borderColor The borderColor to set.
+ */
+ public void setBorderColor(Color borderColor)
+ {
+ this.borderColor = borderColor;
+ }
+
+
+
+ /**
+ * @return Returns the displayTime.
+ */
+ public int getDisplayTime()
+ {
+ return displayTime;
+ }
+
+
+
+ /**
+ * @param displayTime The displayTime to set.
+ */
+ public void setDisplayTime(int displayTime)
+ {
+ this.displayTime = displayTime;
+ }
+
+
+
+ /**
+ * @return Returns the margin.
+ */
+ public int getMargin()
+ {
+ return margin;
+ }
+
+
+
+ /**
+ * @param margin The margin to set.
+ */
+ public void setMargin(int margin)
+ {
+ this.margin = margin;
+ }
+
+
+
+ /**
+ * @return Returns the messageColor.
+ */
+ public Color getMessageColor()
+ {
+ return messageColor;
+ }
+
+
+
+ /**
+ * @param messageColor The messageColor to set.
+ */
+ public void setMessageColor(Color messageColor)
+ {
+ this.messageColor = messageColor;
+ }
+
+
+
+ /**
+ * @return Returns the step.
+ */
+ public int getStep()
+ {
+ return step;
+ }
+
+
+
+ /**
+ * @param step The step to set.
+ */
+ public void setStep(int step)
+ {
+ this.step = step;
+ }
+
+
+
+ /**
+ * @return Returns the stepTime.
+ */
+ public int getStepTime()
+ {
+ return stepTime;
+ }
+
+
+
+ /**
+ * @param stepTime The stepTime to set.
+ */
+ public void setStepTime(int stepTime)
+ {
+ this.stepTime = stepTime;
+ }
+
+
+
+ /**
+ * @return Returns the toasterColor.
+ */
+ public Color getToasterColor()
+ {
+ return toasterColor;
+ }
+
+
+
+ /**
+ * @param toasterColor The toasterColor to set.
+ */
+ public void setToasterColor(Color toasterColor)
+ {
+ this.toasterColor = toasterColor;
+ }
+
+
+
+ /**
+ * @return Returns the toasterHeight.
+ */
+ public int getToasterHeight()
+ {
+ return toasterHeight;
+ }
+
+
+
+ /**
+ * @param toasterHeight The toasterHeight to set.
+ */
+ public void setToasterHeight(int toasterHeight)
+ {
+ this.toasterHeight = toasterHeight;
+ }
+
+
+
+ /**
+ * @return Returns the toasterWidth.
+ */
+ public int getToasterWidth()
+ {
+ return toasterWidth;
+ }
+
+
+
+ /**
+ * @param toasterWidth The toasterWidth to set.
+ */
+ public void setToasterWidth(int toasterWidth)
+ {
+ this.toasterWidth = toasterWidth;
+ }
+
+
+
+}
|