This is an automated email from the ASF dual-hosted git repository.
kturner pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/fluo.git
The following commit(s) were added to refs/heads/master by this push:
new 5b30cd3 fixes #893 added ability to secure zookeeper (#899)
5b30cd3 is described below
commit 5b30cd39adab45848ed9964f4c521d493715578a
Author: Keith Turner <[email protected]>
AuthorDate: Mon Aug 14 15:30:06 2017 +0000
fixes #893 added ability to secure zookeeper (#899)
---
.../org/apache/fluo/api/client/FluoFactory.java | 7 -
.../apache/fluo/api/config/FluoConfiguration.java | 37 ++++-
.../org/apache/fluo/core/client/FluoAdminImpl.java | 1 -
.../org/apache/fluo/core/util/CuratorUtil.java | 55 ++++++-
.../src/main/config/fluo-conn.properties | 6 +
.../apache/fluo/integration/impl/ZKSecretIT.java | 175 +++++++++++++++++++++
6 files changed, 263 insertions(+), 18 deletions(-)
diff --git
a/modules/api/src/main/java/org/apache/fluo/api/client/FluoFactory.java
b/modules/api/src/main/java/org/apache/fluo/api/client/FluoFactory.java
index 52b3843..98d260a 100644
--- a/modules/api/src/main/java/org/apache/fluo/api/client/FluoFactory.java
+++ b/modules/api/src/main/java/org/apache/fluo/api/client/FluoFactory.java
@@ -25,8 +25,6 @@ import org.apache.fluo.api.exceptions.FluoException;
import org.apache.fluo.api.mini.MiniFluo;
import org.apache.fluo.api.service.FluoOracle;
import org.apache.fluo.api.service.FluoWorker;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import static org.apache.fluo.api.config.FluoConfiguration.FLUO_PREFIX;
@@ -38,8 +36,6 @@ import static
org.apache.fluo.api.config.FluoConfiguration.FLUO_PREFIX;
*/
public class FluoFactory {
- private static final Logger log = LoggerFactory.getLogger(FluoFactory.class);
-
private static final String FLUO_IMPL_PREFIX = FLUO_PREFIX + ".impl";
private static final String CLIENT_CLASS_PROP = FLUO_IMPL_PREFIX +
".client.class";
private static final String CLIENT_CLASS_DEFAULT =
"org.apache.fluo.core.client.FluoClientImpl";
@@ -123,14 +119,11 @@ public class FluoFactory {
String msg =
"Could not find " + clazz
+ " class which could be caused by fluo-core jar not being on
the classpath.";
- log.error(msg);
throw new FluoException(msg, e);
} catch (InvocationTargetException e) {
String msg = "Failed to construct " + clazz + " class due to exception";
- log.error(msg, e);
throw new FluoException(msg, e);
} catch (Exception e) {
- log.error("Could not instantiate class - " + clazz);
throw new FluoException(e);
}
}
diff --git
a/modules/api/src/main/java/org/apache/fluo/api/config/FluoConfiguration.java
b/modules/api/src/main/java/org/apache/fluo/api/config/FluoConfiguration.java
index 54e07d2..6c66b6f 100644
---
a/modules/api/src/main/java/org/apache/fluo/api/config/FluoConfiguration.java
+++
b/modules/api/src/main/java/org/apache/fluo/api/config/FluoConfiguration.java
@@ -113,10 +113,17 @@ public class FluoConfiguration extends
SimpleConfiguration {
*/
public static final String CONNECTION_ZOOKEEPER_TIMEOUT_PROP =
CONNECTION_PREFIX
+ ".zookeeper.timeout";
+
+ /**
+ * @since 1.2.0
+ */
+ public static final String CONNECTION_ZOOKEEPER_SECRET = CONNECTION_PREFIX +
".zookeeper.secret";
+
/**
* @since 1.2.0
*/
public static final String CONNECTION_ZOOKEEPERS_PROP = CONNECTION_PREFIX +
".zookeepers";
+
/**
* @since 1.2.0
*/
@@ -345,6 +352,33 @@ public class FluoConfiguration extends SimpleConfiguration
{
CONNECTION_ZOOKEEPER_TIMEOUT_DEFAULT);
}
+ /**
+ * Get the secret configured to access data in zookeeper. If the secret is
an empty string, then
+ * nothing in zookeeper is locked down.
+ *
+ * <p>
+ * Gets the value of the property {@value #CONNECTION_ZOOKEEPER_SECRET}
+ *
+ * @since 1.2.0
+ */
+ public String getZookeeperSecret() {
+ return getString(CONNECTION_ZOOKEEPER_SECRET, "");
+ }
+
+ /**
+ * Setting this before initializing an application will cause Fluo to lock
down Zookeeper such
+ * that this secret is required to read data from zookeeper. If set to an
empty string, then
+ * nothing in zookeeper will be locked down. This property defaults to an
empty string.
+ *
+ * <p>
+ * Sets the value of the property {@value #CONNECTION_ZOOKEEPER_SECRET}
+ *
+ * @since 1.2.0
+ */
+ public void setZookeeperSecret(String secret) {
+ setProperty(CONNECTION_ZOOKEEPER_SECRET,
verifyNotNull(CONNECTION_ZOOKEEPER_SECRET, secret));
+ }
+
@Deprecated
public FluoConfiguration setClientRetryTimeout(int timeoutMs) {
return setConnectionRetryTimeout(timeoutMs);
@@ -968,7 +1002,6 @@ public class FluoConfiguration extends SimpleConfiguration
{
}
private static String verifyNotNull(String property, String value) {
- Objects.requireNonNull(value, property + " cannot be null");
- return value;
+ return Objects.requireNonNull(value, property + " cannot be null");
}
}
diff --git
a/modules/core/src/main/java/org/apache/fluo/core/client/FluoAdminImpl.java
b/modules/core/src/main/java/org/apache/fluo/core/client/FluoAdminImpl.java
index 8300863..eb22830 100644
--- a/modules/core/src/main/java/org/apache/fluo/core/client/FluoAdminImpl.java
+++ b/modules/core/src/main/java/org/apache/fluo/core/client/FluoAdminImpl.java
@@ -31,7 +31,6 @@ import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.UUID;
-import java.util.regex.Matcher;
import java.util.regex.Pattern;
import com.google.common.annotations.VisibleForTesting;
diff --git
a/modules/core/src/main/java/org/apache/fluo/core/util/CuratorUtil.java
b/modules/core/src/main/java/org/apache/fluo/core/util/CuratorUtil.java
index 7b649d6..5f5760b 100644
--- a/modules/core/src/main/java/org/apache/fluo/core/util/CuratorUtil.java
+++ b/modules/core/src/main/java/org/apache/fluo/core/util/CuratorUtil.java
@@ -16,10 +16,13 @@
package org.apache.fluo.core.util;
import java.nio.charset.StandardCharsets;
+import java.util.List;
import java.util.concurrent.TimeUnit;
+import com.google.common.collect.ImmutableList;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.curator.framework.recipes.cache.NodeCacheListener;
@@ -32,6 +35,8 @@ import org.apache.fluo.core.impl.Environment;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.data.ACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -50,7 +55,8 @@ public class CuratorUtil {
* at Fluo application chroot.
*/
public static CuratorFramework newAppCurator(FluoConfiguration config) {
- return newCurator(config.getAppZookeepers(), config.getZookeeperTimeout());
+ return newCurator(config.getAppZookeepers(), config.getZookeeperTimeout(),
+ config.getZookeeperSecret());
}
/**
@@ -58,7 +64,8 @@ public class CuratorUtil {
* chroot.
*/
public static CuratorFramework newFluoCurator(FluoConfiguration config) {
- return newCurator(config.getInstanceZookeepers(),
config.getZookeeperTimeout());
+ return newCurator(config.getInstanceZookeepers(),
config.getZookeeperTimeout(),
+ config.getZookeeperSecret());
}
/**
@@ -67,15 +74,47 @@ public class CuratorUtil {
*/
public static CuratorFramework newRootFluoCurator(FluoConfiguration config) {
return
newCurator(ZookeeperUtil.parseServers(config.getInstanceZookeepers()),
- config.getZookeeperTimeout());
+ config.getZookeeperTimeout(), config.getZookeeperSecret());
}
+ private static final List<ACL> CREATOR_ALL_ACL = ImmutableList.of(new
ACL(Perms.ALL,
+ ZooDefs.Ids.AUTH_IDS));
+
+ private static final List<ACL> PUBLICLY_READABLE_ACL = ImmutableList.of(new
ACL(Perms.READ,
+ ZooDefs.Ids.ANYONE_ID_UNSAFE), new ACL(Perms.ALL, ZooDefs.Ids.AUTH_IDS));
+
/**
* Creates a curator built using the given zookeeper connection string and
timeout
*/
- public static CuratorFramework newCurator(String zookeepers, int timeout) {
- return CuratorFrameworkFactory.newClient(zookeepers, timeout, timeout,
- new ExponentialBackoffRetry(1000, 10));
+ public static CuratorFramework newCurator(String zookeepers, int timeout,
String secret) {
+
+ final ExponentialBackoffRetry retry = new ExponentialBackoffRetry(1000,
10);
+
+ if (secret.isEmpty()) {
+ return CuratorFrameworkFactory.newClient(zookeepers, timeout, timeout,
retry);
+ } else {
+ return CuratorFrameworkFactory.builder().connectString(zookeepers)
+
.connectionTimeoutMs(timeout).sessionTimeoutMs(timeout).retryPolicy(retry)
+ .authorization("digest", ("fluo:" +
secret).getBytes(StandardCharsets.UTF_8))
+ .aclProvider(new ACLProvider() {
+ @Override
+ public List<ACL> getDefaultAcl() {
+ return CREATOR_ALL_ACL;
+ }
+
+ @Override
+ public List<ACL> getAclForPath(String path) {
+ switch (path) {
+ case ZookeeperPath.ORACLE_GC_TIMESTAMP:
+ // The garbage collection iterator running in Accumulo
tservers needs to read this
+ // value w/o authenticating.
+ return PUBLICLY_READABLE_ACL;
+ default:
+ return CREATOR_ALL_ACL;
+ }
+ }
+ }).build();
+ }
}
public static boolean putData(CuratorFramework curator, String zPath, byte[]
data,
@@ -87,7 +126,7 @@ public class CuratorUtil {
while (true) {
try {
curator.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
- .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(zPath, data);
+ .forPath(zPath, data);
return true;
} catch (Exception nee) {
if (nee instanceof KeeperException.NodeExistsException) {
@@ -119,7 +158,7 @@ public class CuratorUtil {
/**
* Starts the ephemeral node and waits for it to be created
- *
+ *
* @param node Node to start
* @param maxWaitSec Maximum time in seconds to wait
*/
diff --git a/modules/distribution/src/main/config/fluo-conn.properties
b/modules/distribution/src/main/config/fluo-conn.properties
index 52db6b2..36a39b4 100644
--- a/modules/distribution/src/main/config/fluo-conn.properties
+++ b/modules/distribution/src/main/config/fluo-conn.properties
@@ -28,3 +28,9 @@
#fluo.connection.zookeeper.timeout=30000
## Connection retry timeout (in milliseconds). Set to -1 to retry forever.
#fluo.connection.retry.timeout.ms=-1
+## Set this property before initialization and Fluo will lock down all
zookeeper
+## using this secret. After initialization, the same secret must always be
+## supplied for any connection or services. If not set, then all data in
+## zookeeper will be world readable. It's a good idea to set this property.
+## Could use the command : head -c 9 /dev/random | base64
+#fluo.connection.zookeeper.secret=
diff --git
a/modules/integration/src/test/java/org/apache/fluo/integration/impl/ZKSecretIT.java
b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ZKSecretIT.java
new file mode 100644
index 0000000..0cc4ac1
--- /dev/null
+++
b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ZKSecretIT.java
@@ -0,0 +1,175 @@
+/*
+ * 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.fluo.integration.impl;
+
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.fluo.accumulo.util.ZookeeperPath;
+import org.apache.fluo.api.client.FluoClient;
+import org.apache.fluo.api.client.FluoFactory;
+import org.apache.fluo.api.client.Snapshot;
+import org.apache.fluo.api.client.Transaction;
+import org.apache.fluo.api.config.FluoConfiguration;
+import org.apache.fluo.api.config.SimpleConfiguration;
+import org.apache.fluo.api.data.Column;
+import org.apache.fluo.api.observer.ObserverProvider;
+import org.apache.fluo.integration.ITBaseMini;
+import org.apache.zookeeper.KeeperException.NoAuthException;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.fluo.api.observer.Observer.NotificationType.STRONG;
+
+public class ZKSecretIT extends ITBaseMini {
+
+ public static class MyObserverProvider implements ObserverProvider {
+
+ @Override
+ public void provide(Registry or, Context ctx) {
+ or.forColumn(new Column("edge", "forward"), STRONG).useObserver((tx,
row, col) -> {
+ tx.set(tx.get(row, col), new Column("edge", "reverese"), row);
+ });
+ }
+
+ }
+
+ @Override
+ protected void setConfig(FluoConfiguration config) {
+ config.setZookeeperSecret("are3");
+ config.setObserverProvider(MyObserverProvider.class);
+ }
+
+ private ZooKeeper getZookeeper() throws IOException {
+ ZooKeeper zk = new ZooKeeper(config.getAppZookeepers(), 30000, null);
+
+ long start = System.currentTimeMillis();
+ while (!zk.getState().isConnected() && System.currentTimeMillis() - start
< 30000) {
+ Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS);
+ }
+
+ return zk;
+ }
+
+ @Test
+ public void testClientWithoutZKSecret() {
+ try (FluoClient client =
FluoFactory.newClient(miniFluo.getClientConfiguration())) {
+ try (Transaction tx = client.newTransaction()) {
+ tx.set("node08", new Column("edge", "forward"), "node75");
+ tx.commit();
+ }
+
+ miniFluo.waitForObservers();
+ }
+
+ FluoConfiguration conf = new
FluoConfiguration(miniFluo.getClientConfiguration());
+ conf.setZookeeperSecret("");
+ try (FluoClient client = FluoFactory.newClient(conf)) {
+ Assert.fail("Expected client creation to fail.");
+ } catch (Exception e) {
+ boolean sawNoAuth = false;
+ Throwable throwable = e;
+ while (throwable != null) {
+ if (throwable instanceof NoAuthException) {
+ sawNoAuth = true;
+ break;
+ }
+ throwable = throwable.getCause();
+ }
+
+ Assert.assertTrue(sawNoAuth);
+ }
+
+ }
+
+ @Test
+ public void testZKAcls() throws Exception {
+
+ // verify basic functionality works when password is set in ZK
+ try (FluoClient client =
FluoFactory.newClient(miniFluo.getClientConfiguration())) {
+ try (Transaction tx = client.newTransaction()) {
+ tx.set("node08", new Column("edge", "forward"), "node75");
+ tx.commit();
+ }
+
+ miniFluo.waitForObservers();
+
+ try (Snapshot snap = client.newSnapshot()) {
+ Assert.assertEquals("node08", snap.gets("node75", new Column("edge",
"reverese")));
+ }
+ }
+
+ ZooKeeper zk = getZookeeper();
+
+
+ // Verify oracle gc timestamp is visible w/o a password. The GC iterator
that runs in Accumulo
+ // tablet servers reads this.
+ String ts =
+ new String(zk.getData(ZookeeperPath.ORACLE_GC_TIMESTAMP, false, null),
+ StandardCharsets.UTF_8);
+ Assert.assertTrue(ts.matches("\\d+"));
+
+ // the timestamp should be read only... trying to modify it should fail
+ try {
+ zk.delete(ZookeeperPath.ORACLE_GC_TIMESTAMP, -1);
+ Assert.fail();
+ } catch (NoAuthException nae) {
+ }
+
+ try {
+ zk.setData(ZookeeperPath.ORACLE_GC_TIMESTAMP, "foo".getBytes(), -1);
+ Assert.fail();
+ } catch (NoAuthException nae) {
+ }
+
+ // try accessing a few random nodes in ZK... All should fail.
+ for (String path : Arrays.asList(ZookeeperPath.ORACLE_SERVER,
ZookeeperPath.CONFIG_SHARED,
+ ZookeeperPath.CONFIG_FLUO_OBSERVERS2, ZookeeperPath.TRANSACTOR_NODES))
{
+
+ try {
+ zk.getData(path, false, null);
+ Assert.fail();
+ } catch (NoAuthException nae) {
+ }
+
+
+ try {
+ zk.getChildren(path, false);
+ } catch (NoAuthException nae) {
+ }
+
+ try {
+ zk.delete(path, -1);
+ Assert.fail();
+ } catch (NoAuthException nae) {
+ }
+
+ try {
+ zk.setData(path, "foo".getBytes(), -1);
+ Assert.fail();
+ } catch (NoAuthException nae) {
+ }
+ }
+
+ zk.close();
+
+ }
+}
--
To stop receiving notification emails like this one, please contact
['"[email protected]" <[email protected]>'].