You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by zh...@apache.org on 2017/08/24 09:59:57 UTC

[bookkeeper] branch master updated: ISSUE #337: Docker image: K8s for bookkeeper and instructions on usage

This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 51c660e  ISSUE #337: Docker image: K8s for bookkeeper and instructions on usage
51c660e is described below

commit 51c660ec87a541a4827742569da72a649e2ed5c7
Author: Jia Zhai <zh...@apache.org>
AuthorDate: Thu Aug 24 17:59:45 2017 +0800

    ISSUE #337: Docker image: K8s for bookkeeper and instructions on usage
    
    Descriptions of the changes in this PR:
    
    K8s deployment for bookkeeper and instructions on usage
    
    Author: Jia Zhai <zh...@apache.org>
    
    Reviewers: Luc Perkins <lu...@gmail.com>
    
    This closes #461 from zhaijack/issue_337, closes #337
---
 deploy/kubernetes/gke/bookkeeper.yaml     | 144 +++++++++++++++++++++++
 deploy/kubernetes/gke/zookeeper.yaml      | 184 ++++++++++++++++++++++++++++++
 site/docs/latest/deployment/kubernetes.md | 161 +++++++++++++++++++++++++-
 3 files changed, 488 insertions(+), 1 deletion(-)

diff --git a/deploy/kubernetes/gke/bookkeeper.yaml b/deploy/kubernetes/gke/bookkeeper.yaml
new file mode 100644
index 0000000..e001740
--- /dev/null
+++ b/deploy/kubernetes/gke/bookkeeper.yaml
@@ -0,0 +1,144 @@
+#
+# 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.
+#
+
+## A reference of https://github.com/apache/incubator-pulsar/blob/master/kubernetes/google-container-engine/bookie.yaml
+
+apiVersion: v1
+kind: ConfigMap
+metadata:
+    name: bookie-config
+data:
+    BK_BOOKIE_EXTRA_OPTS: "\"-Xms1g -Xmx1g -XX:MaxDirectMemorySize=1g -XX:+UseG1GC  -XX:MaxGCPauseMillis=10 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB\""
+    BK_bookiePort: "3181"
+    BK_journalDirectory: "/data/bookkeeper/journal"
+    BK_ledgerDirectories: "/bookkeeper/data/ledgers"
+    BK_indexDirectories: "/bookkeeper/data/ledgers"
+    BK_zkServers: zk-0.zookeeper,zk-1.zookeeper,zk-2.zookeeper
+    # TODO: Issue 458: https://github.com/apache/bookkeeper/issues/458
+    #BK_statsProviderClass: org.apache.bookkeeper.stats.PrometheusMetricsProvider
+---
+
+## BookKeeper servers need to access the local disks and the pods
+## cannot be moved across different nodes.
+## For this reason, we run BK as a daemon set, one for each node in the
+## cluster, unless restricted by label selectors
+apiVersion: extensions/v1beta1
+kind: DaemonSet
+metadata:
+    name: bookie
+    labels:
+        app: bookkeeper
+        component: bookie
+spec:
+    template:
+        metadata:
+            labels:
+                app: bookkeeper
+                component: bookie
+                # Specify cluster to allow aggregation by cluster in
+                # the metrics
+                cluster: bookkeeper-gke
+            annotations:
+                prometheus.io/scrape: "true"
+                prometheus.io/port: "8000"
+
+        spec:
+            containers:
+              - name: bookie
+                image: apache/bookkeeper:latest
+                resources:
+                  requests:
+                    memory: "3Gi"
+                    cpu: "1000m"
+                  limits:
+                    memory: "5Gi"
+                    cpu: "2000m"
+                command: [ "/bin/bash", "/opt/bookkeeper/entrypoint.sh" ]
+                args: ["/opt/bookkeeper/bin/bookkeeper", "bookie"]
+                ports:
+                  - containerPort: 3181
+                    name: client
+                envFrom:
+                  - configMapRef:
+                        name: bookie-config
+
+                volumeMounts:
+                  - name: journal-disk
+                    mountPath: /bookkeeper/data/journal
+                  - name: ledgers-disk
+                    mountPath: /bookkeeper/data/ledgers
+
+            volumes:
+                # Mount local disks
+              - name: journal-disk
+                hostPath:
+                    path: /mnt/disks/ssd0
+              - name: ledgers-disk
+                hostPath:
+                    path: /mnt/disks/ssd1
+
+---
+
+##
+## Define the Bookie headless service
+## In practice, in this case, it is only useful to have a view of
+## all the bookie pods that are present
+##
+apiVersion: v1
+kind: Service
+metadata:
+    name: bookie
+    labels:
+        app: bookkeeper
+        component: bookie
+spec:
+    ports:
+      - port: 3181
+        name: server
+    clusterIP: None
+    selector:
+        app: bookkeeper
+        component: bookie
+
+---
+##
+## Run BookKeeper auto-recovery from a different set of containers
+## Auto-Recovery makes sure to restore the replication factor when any bookie
+## crashes and it's not recovering on its own.
+##
+apiVersion: apps/v1beta1
+kind: Deployment
+metadata:
+    name: bookie-autorecovery
+spec:
+    replicas: 2
+    template:
+        metadata:
+            labels:
+                app: bookkeeper
+                component: bookkeeper-replication
+        spec:
+            containers:
+              - name: replication-worker
+                image: apache/bookkeeper:latest
+                command: [ "/bin/bash", "/opt/bookkeeper/entrypoint.sh" ]
+                args: ["/opt/bookkeeper/bin/bookkeeper", "autorecovery"]
+                envFrom:
+                  - configMapRef:
+                        name: bookie-config
\ No newline at end of file
diff --git a/deploy/kubernetes/gke/zookeeper.yaml b/deploy/kubernetes/gke/zookeeper.yaml
new file mode 100644
index 0000000..12fdfa6
--- /dev/null
+++ b/deploy/kubernetes/gke/zookeeper.yaml
@@ -0,0 +1,184 @@
+#
+# 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.
+#
+
+## A reference of https://github.com/kubernetes/contrib/blob/master/statefulsets/zookeeper/zookeeper.yaml
+
+---
+apiVersion: v1
+kind: Service
+metadata:
+  name: zookeeper
+  labels:
+    app: bookkeeper
+    component: zookeeper
+spec:
+  ports:
+  - port: 2888
+    name: server
+  - port: 3888
+    name: leader-election
+  clusterIP: None
+  selector:
+    app: bookkeeper
+    component: zookeeper
+---
+apiVersion: v1
+kind: ConfigMap
+metadata:
+  name: zk-cm
+data:
+  jvm.heap: "1G"
+  tick: "2000"
+  init: "10"
+  sync: "5"
+  client.cnxns: "60"
+  snap.retain: "3"
+  purge.interval: "0"
+---
+apiVersion: policy/v1beta1
+kind: PodDisruptionBudget
+metadata:
+  name: zk-pdb
+spec:
+  selector:
+    matchLabels:
+      app: zk
+  minAvailable: 2
+---
+apiVersion: apps/v1beta1
+kind: StatefulSet
+metadata:
+  name: zk
+  labels:
+    app: bookkeeper
+    component: zookeeper
+spec:
+  serviceName: zookeeper
+  replicas: 3
+  template:
+    metadata:
+      labels:
+        app: bookkeeper
+        component: zookeeper
+        cluster: bookkeeper-gke
+      annotations:
+        pod.alpha.kubernetes.io/initialized: "true"
+        prometheus.io/scrape: "true"
+        prometheus.io/port: "8080"
+
+    spec:
+      affinity:
+        podAntiAffinity:
+          requiredDuringSchedulingIgnoredDuringExecution:
+            - labelSelector:
+                matchExpressions:
+                  - key: "app"
+                    operator: In
+                    values: 
+                      - zookeeper
+              topologyKey: "kubernetes.io/hostname"
+      containers:
+      - name: k8szk
+        imagePullPolicy: Always
+        image: gcr.io/google_samples/k8szk:v2
+        resources:
+          requests:
+            memory: "1Gi"
+            cpu: "500m"
+        ports:
+        - containerPort: 2181
+          name: client
+        - containerPort: 2888
+          name: server
+        - containerPort: 3888
+          name: leader-election
+        env:
+        - name : ZK_REPLICAS
+          value: "3"
+        - name : ZK_HEAP_SIZE
+          valueFrom:
+            configMapKeyRef:
+                name: zk-cm
+                key: jvm.heap
+        - name : ZK_TICK_TIME
+          valueFrom:
+            configMapKeyRef:
+                name: zk-cm
+                key: tick
+        - name : ZK_INIT_LIMIT
+          valueFrom:
+            configMapKeyRef:
+                name: zk-cm
+                key: init
+        - name : ZK_SYNC_LIMIT
+          valueFrom:
+            configMapKeyRef:
+                name: zk-cm
+                key: tick
+        - name : ZK_MAX_CLIENT_CNXNS
+          valueFrom:
+            configMapKeyRef:
+                name: zk-cm
+                key: client.cnxns
+        - name: ZK_SNAP_RETAIN_COUNT
+          valueFrom:
+            configMapKeyRef:
+                name: zk-cm
+                key: snap.retain
+        - name: ZK_PURGE_INTERVAL
+          valueFrom:
+            configMapKeyRef:
+                name: zk-cm
+                key: purge.interval
+        - name: ZK_CLIENT_PORT
+          value: "2181"
+        - name: ZK_SERVER_PORT
+          value: "2888"
+        - name: ZK_ELECTION_PORT
+          value: "3888"
+        command:
+        - sh
+        - -c
+        - zkGenConfig.sh && zkServer.sh start-foreground
+        readinessProbe:
+          exec:
+            command:
+            - "zkOk.sh"
+          initialDelaySeconds: 10
+          timeoutSeconds: 5
+        livenessProbe:
+          exec:
+            command:
+            - "zkOk.sh"
+          initialDelaySeconds: 10
+          timeoutSeconds: 5
+        volumeMounts:
+        - name: datadir
+          mountPath: /var/lib/zookeeper
+      securityContext:
+        runAsUser: 1000
+        fsGroup: 1000
+  volumeClaimTemplates:
+  - metadata:
+      name: datadir
+    spec:
+      accessModes: [ "ReadWriteOnce" ]
+      resources:
+        requests:
+          storage: 5Gi
diff --git a/site/docs/latest/deployment/kubernetes.md b/site/docs/latest/deployment/kubernetes.md
index f651721..d75919c 100644
--- a/site/docs/latest/deployment/kubernetes.md
+++ b/site/docs/latest/deployment/kubernetes.md
@@ -1,4 +1,163 @@
 ---
-title: Deploying BookKeeper on Kubernetes
+title: Deploying Apache BookKeeper on Kubernetes
+tags: [Kubernetes, Google Container Engine]
 logo: img/kubernetes-logo.png
 ---
+
+Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way.
+
+The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for:
+
+* A three-node ZooKeeper cluster
+* A BookKeeper cluster with a bookie runs on each node.
+
+## Setup on Google Container Engine
+
+To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone.
+
+If you'd like to change the number of bookies,  or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource.
+
+[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE).
+
+### Prerequisites
+
+To get started, you'll need:
+
+* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com)
+* An existing Cloud Platform project
+* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools).
+
+### Create a new Kubernetes cluster
+
+You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more.
+
+As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used  [...]
+
+```bash
+$ gcloud config set compute/zone us-central1-a
+$ gcloud config set project your-project-name
+$ gcloud container clusters create bookkeeper-gke-cluster \
+  --machine-type=n1-standard-8 \
+  --num-nodes=3 \
+  --local-ssd-count=2 \
+  --enable-kubernetes-alpha
+```
+
+By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels).
+
+### Dashboard
+
+You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster:
+
+```bash
+$ gcloud container clusters get-credentials bookkeeper-gke-cluster \
+  --zone=us-central1-a \
+  --project=your-project-name
+$ kubectl proxy
+```
+
+By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying.
+
+When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster:
+
+```bash
+$ kubectl get nodes
+```
+
+If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies.
+
+### ZooKeeper
+
+You *must* deploy ZooKeeper as the first component, as it is a dependency for the others.
+
+```bash
+$ kubectl apply -f zookeeper.yaml
+```
+
+Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time:
+
+```bash
+$ kubectl get pods -l component=zookeeper
+NAME      READY     STATUS             RESTARTS   AGE
+zk-0      1/1       Running            0          18m
+zk-1      1/1       Running            0          17m
+zk-2      0/1       Running            6          15m
+```
+
+This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs.
+
+
+If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the
+remote zookeeper server:
+
+```bash
+$ kubectl port-forward zk-0 2181:2181
+$ zk-shell localhost 2181
+```
+
+### Deploy Bookies
+
+Once ZooKeeper cluster is Running, you can then deploy the bookies.
+
+```bash
+$ kubectl apply -f bookkeeper.yaml
+```
+
+You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`:
+
+```bash
+$ kubectl get pods
+```
+
+While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/
+
+You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster.
+
+```bash
+$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper"
+```
+
+An example output of Dice instance is like this:
+```aidl
+➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper"          
+If you don't see a command prompt, try pressing enter.
+Value = 1, epoch = 5, leading
+Value = 2, epoch = 5, leading
+Value = 1, epoch = 5, leading
+Value = 4, epoch = 5, leading
+Value = 5, epoch = 5, leading
+Value = 4, epoch = 5, leading
+Value = 3, epoch = 5, leading
+Value = 5, epoch = 5, leading
+Value = 3, epoch = 5, leading
+Value = 2, epoch = 5, leading
+Value = 1, epoch = 5, leading
+Value = 4, epoch = 5, leading
+Value = 2, epoch = 5, leading
+```
+
+### Un-Deploy
+
+Delete Demo dice instance
+
+```bash
+$kubectl delete deployment dice      
+```
+
+Delete BookKeeper
+```bash
+$ kubectl delete -f bookkeeper.yaml    
+```
+
+Delete ZooKeeper
+```bash
+$ kubectl delete -f zookeeper.yaml    
+```
+
+Delete cluster
+```bash
+$ gcloud container clusters delete bookkeeper-gke-cluster    
+```
+
+
+

-- 
To stop receiving notification emails like this one, please contact
['"commits@bookkeeper.apache.org" <co...@bookkeeper.apache.org>'].