You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2016/03/26 00:19:13 UTC

[6/8] hbase git commit: HBASE-14854 Read meta location from zk

HBASE-14854 Read meta location from zk


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a51ac256
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a51ac256
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a51ac256

Branch: refs/heads/HBASE-14850
Commit: a51ac256e565db1915f30ab96c4536b8edb8c270
Parents: d518092
Author: Elliott Clark <ec...@apache.org>
Authored: Sat Mar 5 00:09:08 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Fri Mar 25 16:12:13 2016 -0700

----------------------------------------------------------------------
 hbase-native-client/Dockerfile                  |  20 ++-
 hbase-native-client/bin/start-docker.sh         |   5 +-
 hbase-native-client/core/BUCK                   | 106 +++++++-------
 .../core/HBaseNativeClientTestEnv.cc            |  42 ------
 .../core/SampleNativeClientTest.cc              |  28 ----
 hbase-native-client/core/location-cache-test.cc |  14 ++
 hbase-native-client/core/location-cache.cc      |  67 +++++++++
 hbase-native-client/core/location-cache.h       |  35 +++++
 .../core/native-client-test-env.cc              |  42 ++++++
 .../core/simple-native-client-test.cc           |  28 ++++
 hbase-native-client/core/test_env.h             |   2 +
 hbase-native-client/if/BUCK                     |   1 +
 hbase-native-client/third-party/BUCK            | 138 ++++++++++---------
 13 files changed, 339 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/Dockerfile
----------------------------------------------------------------------
diff --git a/hbase-native-client/Dockerfile b/hbase-native-client/Dockerfile
index 1364d22..36959a5 100644
--- a/hbase-native-client/Dockerfile
+++ b/hbase-native-client/Dockerfile
@@ -15,7 +15,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-FROM cpp_update 
+FROM pjameson/buck-folly-watchman
 
 ARG CC=/usr/bin/gcc-5
 ARG CXX=/usr/bin/g++-5
@@ -25,20 +25,26 @@ ARG CXXFLAGS="-D_GLIBCXX_USE_CXX11_ABI=0 -fPIC -g -fno-omit-frame-pointer -O3 -p
 RUN apt-get install -y clang-format-3.7 vim maven inetutils-ping
 RUN git clone --depth 1 --branch v2.6.1 https://github.com/google/protobuf.git /usr/src/protobuf && \
   cd /usr/src/protobuf/ && \
+  ldconfig && \
   ./autogen.sh && \
-  ./configure --disable-shared && \
+  ./configure && \
   make && \
-  make check && \
-  make install
+  make install && \ 
+  make clean && \
+  rm -rf .git
+
 RUN cd /usr/src && \
   wget http://www-us.apache.org/dist/zookeeper/zookeeper-3.4.8/zookeeper-3.4.8.tar.gz && \ 
   tar zxf zookeeper-3.4.8.tar.gz && \ 
   rm -rf zookeeper-3.4.8.tar.gz && \
   cd zookeeper-3.4.8 && \
   cd src/c && \
-  ./configure --disable-shared && \
+  ldconfig && \
+  ./configure && \
   make && \
   make install && \
-  make clean 
+  make clean
+
+RUN ldconfig
 
-WORKDIR /usr/local/src/hbase/hbase-native-client
+WORKDIR /usr/src/hbase/hbase-native-client

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/bin/start-docker.sh
----------------------------------------------------------------------
diff --git a/hbase-native-client/bin/start-docker.sh b/hbase-native-client/bin/start-docker.sh
index 4426705..725ed6a 100755
--- a/hbase-native-client/bin/start-docker.sh
+++ b/hbase-native-client/bin/start-docker.sh
@@ -19,8 +19,11 @@
 set -e
 set -x
 
+# Try out some standard docker machine names that could work
 eval "$(docker-machine env docker-vm)"
 eval "$(docker-machine env dinghy)"
+
+# Build the image
 docker build -t hbase_native .
 
 
@@ -36,6 +39,6 @@ fi;
 
 docker run -p 16010:16010/tcp \
            -e "JAVA_HOME=/usr/lib/jvm/java-8-oracle" \
-           -v ${PWD}/..:/usr/local/src/hbase \
+           -v ${PWD}/..:/usr/src/hbase \
            -v ~/.m2:/root/.m2 \
            -it hbase_native  /bin/bash

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index ef027a1..817b5a0 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -15,52 +15,62 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+cxx_library(name="core",
+            headers=[
+                "admin.h",
+                "client.h",
+                "connection.h",
+                "connection_attr.h",
+                "delete.h",
+                "get.h",
+                "hbase_macros.h",
+                "mutation.h",
+                "put.h",
+                "scanner.h",
+                "location-cache.h",
+            ],
+            srcs=[
+                "admin.cc",
+                "client.cc",
+                "connection.cc",
+                "get.cc",
+                "mutation.cc",
+                "put.cc",
+                "delete.cc",
+                "scanner.cc",
+                "location-cache.cc",
+            ],
+            deps=[
+                "//if:if",
+                "//third-party:zookeeper_mt",
+                "//third-party:folly",
+                "//third-party:wangle",
+            ],
+            visibility=[
+                'PUBLIC',
+            ], )
 
-cxx_binary(
-	name = "core",
-  headers = [
-          "admin.h",
-          "client.h",
-          "connection.h",
-          "connection_attr.h",
-          "delete.h",
-          "get.h",
-          "hbase_macros.h",
-          "mutation.h",
-          "put.h",
-          "scanner.h",
-  ],
-	srcs = [
-          "admin.cc",
-          "client.cc",
-          "connection.cc",
-          "get.cc",
-          "mutation.cc",
-          "put.cc",
-          "delete.cc",
-          "scanner.cc",
-	],
-	deps = [
-		"//if:if",
-		"//third-party:folly",
-		"//third-party:wangle",
-	],
-  visibility = [
-    'PUBLIC',
-  ],
-)
-
-cxx_test(
-  name = "core_test",
-  headers = [
-    "test_env.h",
-  ],
-  srcs = [
-    "HBaseNativeClientTestEnv.cc",
-    "SampleNativeClientTest.cc",
-  ],
-  deps = [
-    ":core",
-  ],
-  run_test_separately = True,
-)
+cxx_test(name="simple-test",
+         headers=[
+             "test_env.h",
+         ],
+         srcs=[
+             "native-client-test-env.cc",
+             "simple-native-client-test.cc",
+         ],
+         deps=[
+             ":core",
+         ],
+         run_test_separately=True, )
+cxx_test(name="location-cache-test",
+         headers=[
+             "test_env.h",
+         ],
+         srcs=[
+             "native-client-test-env.cc",
+             "location-cache-test.cc",
+         ],
+         deps=[
+             ":core",
+         ],
+         run_test_separately=True, )

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/HBaseNativeClientTestEnv.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/HBaseNativeClientTestEnv.cc b/hbase-native-client/core/HBaseNativeClientTestEnv.cc
deleted file mode 100644
index b8cb8db..0000000
--- a/hbase-native-client/core/HBaseNativeClientTestEnv.cc
+++ /dev/null
@@ -1,42 +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.
- *
- */
-
-#include <gtest/gtest.h>
-#include <core/test_env.h>
-
-namespace {
-
-class HBaseNativeClientTestEnv : public ::testing::Environment {
- public:
-  void SetUp() override {
-    init_test_env();
-  }
-
-  void TearDown() override {
-    clean_test_env();
-  }
-};
-
-}  // anonymous
-
-int main(int argc, char** argv) {
-  testing::InitGoogleTest(&argc, argv);
-  ::testing::AddGlobalTestEnvironment(new HBaseNativeClientTestEnv());
-  return RUN_ALL_TESTS();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/SampleNativeClientTest.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/SampleNativeClientTest.cc b/hbase-native-client/core/SampleNativeClientTest.cc
deleted file mode 100644
index ef564f7..0000000
--- a/hbase-native-client/core/SampleNativeClientTest.cc
+++ /dev/null
@@ -1,28 +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.
- *
- */
-
-#include "gtest/gtest.h"
-
-/**
- * Sample test.
- */
-TEST(SampleTest, sample) {
-  EXPECT_TRUE(true);
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/location-cache-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/location-cache-test.cc b/hbase-native-client/core/location-cache-test.cc
new file mode 100644
index 0000000..3106e36
--- /dev/null
+++ b/hbase-native-client/core/location-cache-test.cc
@@ -0,0 +1,14 @@
+#include <gtest/gtest.h>
+#include <folly/Memory.h>
+#include <wangle/concurrent/GlobalExecutor.h>
+
+#include "location-cache.h"
+using namespace hbase;
+
+TEST(LocationCacheTest, TestGetMetaNodeContents) {
+  // TODO(elliott): need to make a test utility for this.
+  LocationCache cache{"localhost:2181", wangle::getCPUExecutor()};
+  auto result = cache.LocateMeta();
+  result.wait();
+  ASSERT_FALSE(result.hasException());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/location-cache.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/location-cache.cc b/hbase-native-client/core/location-cache.cc
new file mode 100644
index 0000000..cf61e24
--- /dev/null
+++ b/hbase-native-client/core/location-cache.cc
@@ -0,0 +1,67 @@
+#include "location-cache.h"
+
+#include <folly/Logging.h>
+
+#include "if/ZooKeeper.pb.h"
+
+using namespace std;
+using namespace folly;
+using namespace hbase::pb;
+
+namespace hbase {
+
+// TODO(elliott): make this configurable on client creation
+const static string META_LOCATION = "/hbase/meta-region-server";
+
+LocationCache::LocationCache(string quorum_spec,
+                             shared_ptr<folly::Executor> executor)
+    : quorum_spec_(quorum_spec), executor_(executor), meta_promise_(nullptr) {
+  zk_ = zookeeper_init(quorum_spec.c_str(), nullptr, 1000, 0, 0, 0);
+}
+
+LocationCache::~LocationCache() {
+  zookeeper_close(zk_);
+  zk_ = nullptr;
+  LOG(INFO) << "Closed connection to ZooKeeper.";
+}
+
+Future<ServerName> LocationCache::LocateMeta() {
+  lock_guard<mutex> g(meta_lock_);
+  if (meta_promise_ == nullptr) {
+    this->RefreshMetaLocation();
+  }
+  return meta_promise_->getFuture();
+}
+
+void LocationCache::InvalidateMeta() {
+  if (meta_promise_ != nullptr) {
+    lock_guard<mutex> g(meta_lock_);
+    meta_promise_ = nullptr;
+  }
+}
+
+/// MUST hold the meta_lock_
+void LocationCache::RefreshMetaLocation() {
+  meta_promise_ = make_unique<SharedPromise<ServerName>>();
+  executor_->add([&] {
+    meta_promise_->setWith([&] { return this->ReadMetaLocation(); });
+  });
+}
+
+ServerName LocationCache::ReadMetaLocation() {
+  char contents[4096];
+  int len = sizeof(contents);
+  // TODO(elliott): handle disconnects/reconntion as needed.
+  int zk_result =
+      zoo_get(this->zk_, META_LOCATION.c_str(), 0, contents, &len, nullptr);
+
+  if (zk_result != ZOK) {
+    LOG(ERROR) << "Error getting meta location.";
+    throw runtime_error("Error getting meta location");
+  }
+
+  MetaRegionServer mrs;
+  mrs.ParseFromArray(contents, len);
+  return mrs.server();
+}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/location-cache.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/location-cache.h b/hbase-native-client/core/location-cache.h
new file mode 100644
index 0000000..8dc2760
--- /dev/null
+++ b/hbase-native-client/core/location-cache.h
@@ -0,0 +1,35 @@
+#pragma once
+
+#include <memory>
+#include <mutex>
+
+#include <zookeeper/zookeeper.h>
+#include <folly/futures/Future.h>
+#include <folly/futures/SharedPromise.h>
+
+#include <folly/Executor.h>
+#include "if/HBase.pb.h"
+
+namespace hbase {
+class LocationCache {
+public:
+  explicit LocationCache(std::string quorum_spec,
+                         std::shared_ptr<folly::Executor> executor);
+  ~LocationCache();
+  // Meta Related Methods.
+  // These are only public until testing is complete
+  folly::Future<hbase::pb::ServerName> LocateMeta();
+  void InvalidateMeta();
+
+private:
+  void RefreshMetaLocation();
+  hbase::pb::ServerName ReadMetaLocation();
+
+  std::string quorum_spec_;
+  std::shared_ptr<folly::Executor> executor_;
+  std::unique_ptr<folly::SharedPromise<hbase::pb::ServerName>> meta_promise_;
+  std::mutex meta_lock_;
+
+  zhandle_t *zk_;
+};
+} // hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/native-client-test-env.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/native-client-test-env.cc b/hbase-native-client/core/native-client-test-env.cc
new file mode 100644
index 0000000..a86961f
--- /dev/null
+++ b/hbase-native-client/core/native-client-test-env.cc
@@ -0,0 +1,42 @@
+/*
+ * 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 <gtest/gtest.h>
+#include <core/test_env.h>
+
+namespace {
+
+class NativeClientTestEnv : public ::testing::Environment {
+ public:
+  void SetUp() override {
+    init_test_env();
+  }
+
+  void TearDown() override {
+    clean_test_env();
+  }
+};
+
+}  // anonymous
+
+int main(int argc, char** argv) {
+  testing::InitGoogleTest(&argc, argv);
+  ::testing::AddGlobalTestEnvironment(new NativeClientTestEnv());
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/simple-native-client-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/simple-native-client-test.cc b/hbase-native-client/core/simple-native-client-test.cc
new file mode 100644
index 0000000..ef564f7
--- /dev/null
+++ b/hbase-native-client/core/simple-native-client-test.cc
@@ -0,0 +1,28 @@
+/*
+ * 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 "gtest/gtest.h"
+
+/**
+ * Sample test.
+ */
+TEST(SampleTest, sample) {
+  EXPECT_TRUE(true);
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/core/test_env.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/test_env.h b/hbase-native-client/core/test_env.h
index 5796ae1..79bdbec 100644
--- a/hbase-native-client/core/test_env.h
+++ b/hbase-native-client/core/test_env.h
@@ -17,6 +17,8 @@
  *
  */
 
+#pragma once
+
 #include <cstdlib>
 
 inline void init_test_env() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/if/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/BUCK b/hbase-native-client/if/BUCK
index 3490a05..9b989b5 100644
--- a/hbase-native-client/if/BUCK
+++ b/hbase-native-client/if/BUCK
@@ -33,4 +33,5 @@ cxx_library(
         + [':' + x for x in CC_FILENAMES] 
         + [ ':' + x for x in HEADER_FILENAMES ],
   visibility = [ 'PUBLIC', ],
+  exported_deps = ['//third-party:protobuf']
 )

http://git-wip-us.apache.org/repos/asf/hbase/blob/a51ac256/hbase-native-client/third-party/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/third-party/BUCK b/hbase-native-client/third-party/BUCK
index e577a5f..6548695 100644
--- a/hbase-native-client/third-party/BUCK
+++ b/hbase-native-client/third-party/BUCK
@@ -15,85 +15,97 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-def add_system_libs(names = [], lib_dir = "/usr/lib/x86_64-linux-gnu", deps = [], exported_linker_flags = []):
-        rules = []
-        for name in names:
-            gen_rule_name = "gen_lib{}".format(name)
-            genrule(
-              name = gen_rule_name,
-              out = gen_rule_name,
-              bash = "mkdir -p $OUT && cp {}/lib{}.* $OUT".format(lib_dir,name),
-            )
-            prebuilt_cxx_library(
-              name = name,
-              lib_name = name,
-              lib_dir = '$(location :{})'.format(gen_rule_name),
-              force_static = True,
-              deps = deps,
-              visibility = [ 'PUBLIC' ],
-              exported_linker_flags = exported_linker_flags,
-            )
-            rules.append(":" + name)
-        return rules
 
-system_libs = [
-        "unwind",
-        "lzma",
-]
+def add_system_libs(names=[],
+                    lib_dir="/usr/lib/x86_64-linux-gnu",
+                    deps=[],
+                    exported_deps=[],
+                    exported_linker_flags=[]):
+    rules = []
+    for name in names:
+        gen_rule_name = "gen_lib{}".format(name)
+        genrule(name=gen_rule_name,
+                out=gen_rule_name,
+                bash="mkdir -p $OUT && cp {}/lib{}.a $OUT".format(lib_dir,
+                                                                   name), )
+        prebuilt_cxx_library(name=name,
+                             lib_name=name,
+                             lib_dir='$(location :{})'.format(gen_rule_name),
+                             deps=deps,
+			     force_static = True,
+                             exported_deps=exported_deps,
+                             visibility=['PUBLIC'],
+                             exported_linker_flags=exported_linker_flags, )
+        rules.append(":" + name)
+    return rules
+
+
+system_libs = ["unwind", "lzma", "event", ]
 local_libs = [
-        "double-conversion",
-        "glog",
-        "gflags",
-        "protobuf",
-        "zookeeper_mt",
-        "boost_regex",
+    "double-conversion",
+    "boost_regex",
+    "boost_context",
+    "boost_thread",
+    "boost_system",
+    "boost_filesystem",
+    "boost_program_options",
+    "boost_chrono",
+    "gflags",
+    "glog",
+    "protobuf",
 ]
 
+
+
 tp_dep_rules = add_system_libs(system_libs) \
-  + add_system_libs(local_libs, lib_dir = "/usr/local/lib") 
-folly = add_system_libs(
-  ['folly'], 
-  lib_dir = '/usr/local/lib', 
-  deps = tp_dep_rules, 
-  exported_linker_flags = [ "-pthread", "-lstdc++",]
-  )
-folly_bench = add_system_libs(
-  ['follybenchmark'], 
-  lib_dir = '/usr/local/lib', 
-  deps = tp_dep_rules + folly,
-  exported_linker_flags = [ "-pthread", "-lstdc++",]
-  )
-wangle = add_system_libs(
-  ['wangle'], 
-  lib_dir = '/usr/local/lib', 
-  deps = tp_dep_rules + folly, 
-  exported_linker_flags = [ "-pthread", "-lstdc++",]
-  )
+  + add_system_libs(local_libs, lib_dir = "/usr/local/lib")
+
+zookeeper = add_system_libs(["zookeeper_mt"], lib_dir =  "/usr/local/lib")
+folly = add_system_libs(['folly'],
+                        lib_dir='/usr/local/lib',
+                        exported_deps=tp_dep_rules,
+                        exported_linker_flags=["-pthread",
+                                               "-lstdc++", ])
+folly_bench = add_system_libs(['follybenchmark'],
+                              lib_dir='/usr/local/lib',
+                              exported_deps=tp_dep_rules + folly,
+                              exported_linker_flags=["-pthread",
+                                                     "-lstdc++", ])
+wangle = add_system_libs(['wangle'],
+                         lib_dir='/usr/local/lib',
+                         exported_deps=tp_dep_rules + folly,
+                         exported_linker_flags=["-pthread",
+                                                "-lstdc++", ])
+
+
+genrule(
+name = "gen_zk",
+out = "gen_zk",
+bash = "mkdir -p $OUT  && wget http://www-us.apache.org/dist/zookeeper/zookeeper-3.4.8/zookeeper-3.4.8.tar.gz &&   tar zxf zookeeper-3.4.8.tar.gz &&   rm -rf zookeeper-3.4.8.tar.gz &&   cd zookeeper-3.4.8 &&   cd src/c && ./configure --prefix=$OUT &&   make &&   make install && cd $OUT && rm -rf zookeeper-3.4.8*"
+)
 cxx_library(
-  name = 'google-test',
-  srcs = [
+    name = 'google-test',
+    srcs = [
     'googletest/googletest/src/gtest-all.cc',
     'googletest/googlemock/src/gmock-all.cc',
     'googletest/googlemock/src/gmock_main.cc',
-  ], 
-  header_namespace = '',
-  exported_headers = subdir_glob([
+    ],
+    header_namespace = '',
+    exported_headers = subdir_glob([
     ('googletest/googletest/include', '**/*.h'),
     ('googletest/googlemock/include', '**/*.h'),
-  ]),
-  headers = subdir_glob([
+    ]),
+    headers = subdir_glob([
     ('googletest/googletest', 'src/*.h'),
     ('googletest/googletest', 'src/*.cc'),
     ('googletest/googlemock', 'src/*.h'),
     ('googletest/googlemock', 'src/*.cc'),
-  ]),
-  exported_linker_flags = [
+    ]),
+    exported_linker_flags = [
     "-pthread",
     "-lstdc++",
-  ],
-  visibility = [
+    ],
+    visibility = [
     'PUBLIC',
-  ],
-  deps = [
-  ]
+    ],
 )