You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2017/10/12 13:51:16 UTC

kylin git commit: KYLIN-2656 Support Zookeeper ACL

Repository: kylin
Updated Branches:
  refs/heads/master c1c336d65 -> 8891b1c84


KYLIN-2656 Support Zookeeper ACL

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/master
Commit: 8891b1c84ad1fe84199c9c796c0ce57408607435
Parents: c1c336d
Author: peng.jianhua <pe...@zte.com.cn>
Authored: Thu Oct 12 10:50:44 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Oct 12 21:50:52 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |  12 ++
 .../storage/hbase/util/ZookeeperAclBuilder.java | 119 +++++++++++++++++++
 .../hbase/util/ZookeeperDistributedLock.java    |   3 +-
 .../hbase/util/ZookeeperAclBuilderTest.java     |  97 +++++++++++++++
 4 files changed, 230 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8891b1c8/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index f362884..34d8b7c 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -245,6 +245,18 @@ abstract public class KylinConfigBase implements Serializable {
         throw new RuntimeException("Please set 'kylin.env.zookeeper-connect-string' in kylin.properties");
     }
 
+    public boolean isZookeeperAclEnabled() {
+        return Boolean.parseBoolean(getOptional("kylin.env.zookeeper-acl-enabled", "false"));
+    }
+
+    public String getZKAuths() {
+        return getOptional("kylin.env.zookeeper.zk-auth", "digest:ADMIN:KYLIN");
+    }
+
+    public String getZKAcls() {
+        return getOptional("kylin.env.zookeeper.zk-acl", "world:anyone:rwcda");
+    }
+
     // ============================================================================
     // METADATA
     // ============================================================================

http://git-wip-us.apache.org/repos/asf/kylin/blob/8891b1c8/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilder.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilder.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilder.java
new file mode 100644
index 0000000..232521d
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilder.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.util;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.curator.framework.CuratorFrameworkFactory.Builder;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Created by peng.jianhua on 17-6-5.
+ */
+public class ZookeeperAclBuilder {
+
+    private static Logger logger = LoggerFactory.getLogger(ZookeeperAclBuilder.class);
+
+    private List<ACL> zkAcls;
+    private List<ZKUtil.ZKAuthInfo> zkAuthInfo;
+    private boolean isNeedAcl = KylinConfig.getInstanceFromEnv().isZookeeperAclEnabled();
+
+    public Builder setZKAclBuilder(Builder builder) {
+        Builder aclBuilder;
+        ACLProvider aclProvider;
+
+        if (!isNeedAcl()) {
+            return builder;
+        }
+
+        aclProvider = new ACLProvider() {
+            private List<ACL> acl;
+
+            @Override
+            public List<ACL> getDefaultAcl() {
+                if (acl == null) {
+                    this.acl = zkAcls;
+                }
+                return acl;
+            }
+
+            @Override
+            public List<ACL> getAclForPath(String path) {
+                return acl;
+            }
+        };
+
+        aclBuilder = builder.aclProvider(aclProvider);
+        for (ZKUtil.ZKAuthInfo auth : zkAuthInfo) {
+            aclBuilder = aclBuilder.authorization(auth.getScheme(), auth.getAuth());
+        }
+        return aclBuilder;
+    }
+
+    public ZookeeperAclBuilder invoke() {
+        try {
+            if (isNeedAcl()) {
+                zkAcls = getZKAcls();
+                zkAuthInfo = getZKAuths();
+            }
+        } catch (Exception e) {
+            isNeedAcl = false;
+            return this;
+        }
+        return this;
+    }
+
+    public static List<ZKUtil.ZKAuthInfo> getZKAuths() throws Exception {
+        // Parse Auths from configuration.
+        String zkAuthConf = KylinConfig.getInstanceFromEnv().getZKAuths();
+        try {
+            zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
+            if (zkAuthConf != null) {
+                return ZKUtil.parseAuth(zkAuthConf);
+            } else {
+                return Collections.emptyList();
+            }
+        } catch (Exception e) {
+            logger.error("Couldn't read Auth based on 'kylin.env.zookeeper.zk-auth' in kylin.properties");
+            throw e;
+        }
+    }
+
+    public static List<ACL> getZKAcls() throws Exception {
+        // Parse ACLs from configuration.
+        String zkAclConf = KylinConfig.getInstanceFromEnv().getZKAcls();
+        try {
+            zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
+            return ZKUtil.parseACLs(zkAclConf);
+        } catch (Exception e) {
+            logger.error("Couldn't read ACLs based on 'kylin.env.zookeeper.zk-acl' in kylin.properties");
+            throw e;
+        }
+    }
+
+    public boolean isNeedAcl() {
+        return isNeedAcl;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8891b1c8/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java
index 844d9ed..a7d2cb6 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java
@@ -76,7 +76,8 @@ public class ZookeeperDistributedLock implements DistributedLock, JobLock {
                     if (zkClient == null) {
                         RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
                         String zkConnectString = getZKConnectString(config);
-                        zkClient = CuratorFrameworkFactory.newClient(zkConnectString, 120000, 15000, retryPolicy);
+                        ZookeeperAclBuilder zookeeperAclBuilder = new ZookeeperAclBuilder().invoke();
+                        zkClient = zookeeperAclBuilder.setZKAclBuilder(CuratorFrameworkFactory.builder()).connectString(zkConnectString).sessionTimeoutMs(120000).connectionTimeoutMs(15000).retryPolicy(retryPolicy).build();
                         zkClient.start();
                         CACHE.put(config, zkClient);
                         if (CACHE.size() > 1) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/8891b1c8/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilderTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilderTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilderTest.java
new file mode 100644
index 0000000..25f7687
--- /dev/null
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/util/ZookeeperAclBuilderTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.storage.hbase.util;
+
+import static junit.framework.TestCase.fail;
+
+import java.util.List;
+
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.CuratorFrameworkFactory.Builder;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class ZookeeperAclBuilderTest extends LocalFileMetadataTestCase {
+
+    @BeforeClass
+    public static void setupResource() throws Exception {
+        staticCreateTestMetadata();
+    }
+
+    @AfterClass
+    public static void tearDownResource() {
+        cleanAfterClass();
+    }
+
+    @Test
+    public void testAclEnabled() {
+        KylinConfig testConfig = KylinConfig.getInstanceFromEnv();
+        testConfig.setProperty("kylin.env.zookeeper-acl-enabled", "true");
+
+        ZookeeperAclBuilder zookeeperAclBuilder = new ZookeeperAclBuilder().invoke();
+        Assert.assertNotNull(zookeeperAclBuilder);
+        Assert.assertTrue(zookeeperAclBuilder.isNeedAcl());
+
+        List<ACL> zkAcls = Lists.newArrayList();
+        try {
+            zkAcls = ZookeeperAclBuilder.getZKAcls();
+            Assert.assertFalse(zkAcls.isEmpty());
+        } catch (Exception e) {
+            fail("Couldn't read ACLs based on 'kylin.env.zookeeper.zk-acl' in kylin.properties");
+        }
+
+        List<ZKUtil.ZKAuthInfo> zkAuthInfo = Lists.newArrayList();
+        try {
+            zkAuthInfo = ZookeeperAclBuilder.getZKAuths();
+            Assert.assertFalse(zkAuthInfo.isEmpty());
+        } catch (Exception e) {
+            fail("Couldn't read Auth based on 'kylin.env.zookeeper.zk-auth' in kylin.properties");
+        }
+
+        Builder builder = zookeeperAclBuilder.setZKAclBuilder(CuratorFrameworkFactory.builder());
+        Assert.assertNotNull(builder);
+        Assert.assertEquals(zkAcls, builder.getAclProvider().getDefaultAcl());
+        Assert.assertNotNull(builder.getAuthInfos());
+    }
+
+    @Test
+    public void testAclDisabled() {
+        KylinConfig testConfig = KylinConfig.getInstanceFromEnv();
+        testConfig.setProperty("kylin.env.zookeeper-acl-enabled", "false");
+
+        ZookeeperAclBuilder zookeeperAclBuilder = new ZookeeperAclBuilder().invoke();
+        Assert.assertNotNull(zookeeperAclBuilder);
+        Assert.assertFalse(zookeeperAclBuilder.isNeedAcl());
+
+        Builder builder = zookeeperAclBuilder.setZKAclBuilder(CuratorFrameworkFactory.builder());
+        Assert.assertNotNull(builder);
+        Assert.assertEquals(ZooDefs.Ids.OPEN_ACL_UNSAFE, builder.getAclProvider().getDefaultAcl());
+        Assert.assertNull(builder.getAuthInfos());
+    }
+
+}