You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2014/08/17 19:43:21 UTC

svn commit: r1618484 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/security/ src/main/java/org/apache/hadoop/util/ src/test/java/org/apache/hadoop/security/ src/test/java/org/apache/hadoop/util/

Author: arp
Date: Sun Aug 17 17:43:20 2014
New Revision: 1618484

URL: http://svn.apache.org/r1618484
Log:
HADOOP-10335. An ip whilelist based implementation to resolve Sasl properties per connection. (Contributed by Benoy Antony)

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/WhitelistBasedResolver.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CacheableIPList.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CombinedIPWhiteList.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCacheableIPList.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFileBasedIPList.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1618484&r1=1618483&r2=1618484&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Sun Aug 17 17:43:20 2014
@@ -524,6 +524,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-10650. Add ability to specify a reverse ACL (black list) of users
     and groups. (Benoy Antony via Arpit Agarwal)
 
+    HADOOP-10335. An ip whilelist based implementation to resolve Sasl
+    properties per connection. (Benoy Antony via Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/WhitelistBasedResolver.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/WhitelistBasedResolver.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/WhitelistBasedResolver.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/WhitelistBasedResolver.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,149 @@
+/**
+ * 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.hadoop.security;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.security.sasl.Sasl;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SaslPropertiesResolver;
+import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection;
+import org.apache.hadoop.util.CombinedIPWhiteList;
+import org.apache.hadoop.util.StringUtils;
+
+
+/**
+ * An implementation of the SaslPropertiesResolver.
+ * Uses a white list of IPs.
+ * If the connection's IP address is in the list of IP addresses, the salProperties
+ * will be unchanged.
+ * If the connection's IP is not in the list of IP addresses, then QOP for the
+ * connection will be restricted to "hadoop.rpc.protection.non-whitelist"
+ *
+ * Uses 3 IPList implementations together to form an aggregate whitelist.
+ * 1. ConstantIPList - to check against a set of hardcoded IPs
+ * 2. Fixed IP List - to check against a list of IP addresses which are specified externally, but
+ * will not change over runtime.
+ * 3. Variable IP List - to check against a list of IP addresses which are specified externally and
+ * could change during runtime.
+ * A connection IP address will checked against these 3 IP Lists in the order specified above.
+ * Once a match is found , the IP address is determined to be in whitelist.
+ *
+ * The behavior can be configured using a bunch of configuration parameters.
+ *
+ */
+public class WhitelistBasedResolver extends SaslPropertiesResolver {
+  public static final Log LOG = LogFactory.getLog(WhitelistBasedResolver.class);
+
+  private static final String FIXEDWHITELIST_DEFAULT_LOCATION = "/etc/hadoop/fixedwhitelist";
+
+  private static final String VARIABLEWHITELIST_DEFAULT_LOCATION = "/etc/hadoop/whitelist";
+
+  /**
+   * Path to the file to containing subnets and ip addresses to form fixed whitelist.
+   */
+  public static final String HADOOP_SECURITY_SASL_FIXEDWHITELIST_FILE =
+    "hadoop.security.sasl.fixedwhitelist.file";
+  /**
+   * Enables/Disables variable whitelist
+   */
+  public static final String HADOOP_SECURITY_SASL_VARIABLEWHITELIST_ENABLE =
+    "hadoop.security.sasl.variablewhitelist.enable";
+  /**
+   * Path to the file to containing subnets and ip addresses to form variable whitelist.
+   */
+  public static final String HADOOP_SECURITY_SASL_VARIABLEWHITELIST_FILE =
+    "hadoop.security.sasl.variablewhitelist.file";
+  /**
+   * time in seconds by which the variable whitelist file is checked for updates
+   */
+  public static final String HADOOP_SECURITY_SASL_VARIABLEWHITELIST_CACHE_SECS =
+    "hadoop.security.sasl.variablewhitelist.cache.secs";
+
+  /**
+   * comma separated list containing alternate hadoop.rpc.protection values for
+   * clients which are not in whitelist
+   */
+  public static final String HADOOP_RPC_PROTECTION_NON_WHITELIST =
+    "hadoop.rpc.protection.non-whitelist";
+
+  private CombinedIPWhiteList whiteList;
+
+  private Map<String, String> saslProps;
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    String fixedFile = conf.get(HADOOP_SECURITY_SASL_FIXEDWHITELIST_FILE,
+        FIXEDWHITELIST_DEFAULT_LOCATION);
+    String variableFile = null;
+    long expiryTime = 0;
+
+    if (conf.getBoolean(HADOOP_SECURITY_SASL_VARIABLEWHITELIST_ENABLE, false)) {
+      variableFile = conf.get(HADOOP_SECURITY_SASL_VARIABLEWHITELIST_FILE,
+          VARIABLEWHITELIST_DEFAULT_LOCATION);
+      expiryTime =
+        conf.getLong(HADOOP_SECURITY_SASL_VARIABLEWHITELIST_CACHE_SECS,3600) * 1000;
+    }
+
+    whiteList = new CombinedIPWhiteList(fixedFile,variableFile,expiryTime);
+
+    this.saslProps = getSaslProperties(conf);
+  }
+
+  /**
+   * Identify the Sasl Properties to be used for a connection with a client.
+   * @param clientAddress client's address
+   * @return the sasl properties to be used for the connection.
+   */
+  @Override
+  public Map<String, String> getServerProperties(InetAddress clientAddress) {
+    if (clientAddress == null) {
+      return saslProps;
+    }
+    return  whiteList.isIn(clientAddress.getHostAddress())?getDefaultProperties():saslProps;
+  }
+
+  public Map<String, String> getServerProperties(String clientAddress) throws UnknownHostException {
+    if (clientAddress == null) {
+      return saslProps;
+    }
+    return getServerProperties(InetAddress.getByName(clientAddress));
+  }
+
+  static Map<String, String> getSaslProperties(Configuration conf) {
+    Map<String, String> saslProps =new TreeMap<String, String>();
+    String[] qop = conf.getStrings(HADOOP_RPC_PROTECTION_NON_WHITELIST,
+        QualityOfProtection.PRIVACY.toString());
+
+    for (int i=0; i < qop.length; i++) {
+      qop[i] = QualityOfProtection.valueOf(qop[i].toUpperCase()).getSaslQop();
+    }
+
+    saslProps.put(Sasl.QOP, StringUtils.join(",", qop));
+    saslProps.put(Sasl.SERVER_AUTH, "true");
+
+    return saslProps;
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CacheableIPList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CacheableIPList.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CacheableIPList.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CacheableIPList.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,76 @@
+/**
+ * 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.hadoop.util;
+
+/**
+ * CacheableIPList loads a list of subnets from a file.
+ * The list is cached and the cache can be refreshed by specifying cache timeout.
+ * A negative value of cache timeout disables any caching.
+ *
+ * Thread safe.
+ */
+
+public class CacheableIPList implements IPList {
+  private final long cacheTimeout;
+  private volatile long cacheExpiryTimeStamp;
+  private volatile FileBasedIPList ipList;
+
+  public CacheableIPList(FileBasedIPList ipList, long cacheTimeout) {
+    this.cacheTimeout =  cacheTimeout;
+    this.ipList = ipList;
+    updateCacheExpiryTime();
+  }
+
+  /**
+   * Reloads the ip list
+   */
+  private  void  reset() {
+    ipList = ipList.reload();
+    updateCacheExpiryTime();
+  }
+
+  private void updateCacheExpiryTime() {
+    if (cacheTimeout < 0) {
+      cacheExpiryTimeStamp = -1; // no automatic cache expiry.
+    }else {
+      cacheExpiryTimeStamp = System.currentTimeMillis() + cacheTimeout;
+    }
+  }
+
+  /**
+   * Refreshes the ip list
+   */
+  public  void refresh () {
+    cacheExpiryTimeStamp = 0;
+  }
+
+  @Override
+  public boolean isIn(String ipAddress) {
+    //is cache expired
+    //Uses Double Checked Locking using volatile
+    if (cacheExpiryTimeStamp >= 0 && cacheExpiryTimeStamp < System.currentTimeMillis()) {
+      synchronized(this) {
+        //check if cache expired again
+        if (cacheExpiryTimeStamp < System.currentTimeMillis()) {
+          reset();
+        }
+      }
+    }
+    return ipList.isIn(ipAddress);
+  }
+}
\ No newline at end of file

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CombinedIPWhiteList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CombinedIPWhiteList.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CombinedIPWhiteList.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CombinedIPWhiteList.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,60 @@
+/**
+ * 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.hadoop.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class CombinedIPWhiteList implements IPList {
+
+  public static final Log LOG = LogFactory.getLog(CombinedIPWhiteList.class);
+  private static final String LOCALHOST_IP = "127.0.0.1";
+
+  private final IPList[] networkLists;
+
+  public CombinedIPWhiteList(String fixedWhiteListFile,
+      String variableWhiteListFile, long cacheExpiryInSeconds) {
+
+    IPList fixedNetworkList = new FileBasedIPList(fixedWhiteListFile);
+    if (variableWhiteListFile != null){
+      IPList variableNetworkList = new CacheableIPList(
+          new FileBasedIPList(variableWhiteListFile),cacheExpiryInSeconds);
+      networkLists = new IPList[] {fixedNetworkList, variableNetworkList};
+    }
+    else {
+      networkLists = new IPList[] {fixedNetworkList};
+    }
+  }
+  @Override
+  public boolean isIn(String ipAddress) {
+    if (ipAddress == null) {
+      throw new IllegalArgumentException("ipAddress is null");
+    }
+
+    if (LOCALHOST_IP.equals(ipAddress)) {
+      return true;
+    }
+
+    for (IPList networkList:networkLists) {
+      if (networkList.isIn(ipAddress)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,102 @@
+/**
+ * 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.hadoop.util;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * FileBasedIPList loads a list of subnets in CIDR format and ip addresses from a file.
+ *
+ * Given an ip address, isIn  method returns true if ip belongs to one of the subnets.
+ *
+ * Thread safe.
+ */
+
+public class FileBasedIPList implements IPList {
+
+  private static final Log LOG = LogFactory.getLog(FileBasedIPList.class);
+
+  private final String fileName;
+  private final MachineList addressList;
+
+  public FileBasedIPList(String fileName) {
+    this.fileName = fileName;
+    String[] lines = readLines(fileName);
+    if (lines != null) {
+      addressList = new MachineList(new HashSet<String>(Arrays.asList(lines)));
+    } else {
+      addressList = null;
+    }
+  }
+
+  public FileBasedIPList reload() {
+    return new FileBasedIPList(fileName);
+  }
+
+  @Override
+  public  boolean isIn(String ipAddress) {
+    if (ipAddress == null || addressList == null) {
+      return false;
+    }
+    return addressList.includes(ipAddress);
+  }
+
+  /**
+   * reads the lines in a file.
+   * @param fileName
+   * @return lines in a String array; null if the file does not exist or if the
+   * file name is null
+   * @throws IOException
+   */
+  private static String[] readLines(String fileName) {
+    try {
+      if (fileName != null) {
+        File file = new File (fileName);
+        if (file.exists()) {
+          FileReader fileReader = new FileReader(file);
+          BufferedReader bufferedReader = new BufferedReader(fileReader);
+          List<String> lines = new ArrayList<String>();
+          String line = null;
+          while ((line = bufferedReader.readLine()) != null) {
+            lines.add(line);
+          }
+          bufferedReader.close();
+          LOG.debug("Loaded IP list of size = " + lines.size() +" from file = " + fileName);
+          return(lines.toArray(new String[lines.size()]));
+        }
+        else {
+          LOG.debug("Missing ip list file : "+ fileName);
+        }
+      }
+    }
+    catch (Throwable t) {
+      LOG.error(t);
+    }
+    return null;
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceStability.Unstable
+@InterfaceAudience.Public
+public interface IPList {
+
+  /**
+   * returns true if the ipAddress is in the IPList.
+   * @param ipAddress
+   * @return boolean value indicating whether the ipAddress is in the IPList
+   */
+  public abstract boolean isIn(String ipAddress);
+}
\ No newline at end of file

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java?rev=1618484&r1=1618483&r2=1618484&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java Sun Aug 17 17:43:20 2014
@@ -37,7 +37,7 @@ import com.google.common.net.InetAddress
 /**
  * Container class which holds a list of ip/host addresses and 
  * answers membership queries.
- * .
+ *
  * Accepts list of ip addresses, ip addreses in CIDR format and/or 
  * host addresses.
  */
@@ -71,8 +71,15 @@ public class MachineList {
    * @param hostEntries comma separated ip/cidr/host addresses
    */
   public MachineList(String hostEntries) {
-    this(StringUtils.getTrimmedStringCollection(hostEntries),
-        InetAddressFactory.S_INSTANCE);
+    this(StringUtils.getTrimmedStringCollection(hostEntries));
+  }
+
+  /**
+   *
+   * @param hostEntries collection of separated ip/cidr/host addresses
+   */
+  public MachineList(Collection<String> hostEntries) {
+    this(hostEntries, InetAddressFactory.S_INSTANCE);
   }
 
   /**

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,163 @@
+/**
+ * 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.hadoop.security;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.WhitelistBasedResolver;
+import org.apache.hadoop.util.TestFileBasedIPList;
+
+public class TestWhitelistBasedResolver extends TestCase {
+
+  public static final Map<String, String> SASL_PRIVACY_PROPS =
+    WhitelistBasedResolver.getSaslProperties(new Configuration());
+
+  public void testFixedVariableAndLocalWhiteList() throws IOException {
+
+    String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
+
+    TestFileBasedIPList.createFileWithEntries ("fixedwhitelist.txt", fixedIps);
+
+    String[] variableIps = {"10.222.0.0/16", "10.113.221.221"};
+
+    TestFileBasedIPList.createFileWithEntries ("variablewhitelist.txt", variableIps);
+
+    Configuration conf = new Configuration();
+    conf.set(WhitelistBasedResolver.HADOOP_SECURITY_SASL_FIXEDWHITELIST_FILE ,
+        "fixedwhitelist.txt");
+
+    conf.setBoolean(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_ENABLE,
+        true);
+
+    conf.setLong(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_CACHE_SECS,
+        1);
+
+    conf.set(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_FILE ,
+        "variablewhitelist.txt");
+
+    WhitelistBasedResolver wqr = new WhitelistBasedResolver ();
+    wqr.setConf(conf);
+
+    assertEquals (wqr.getDefaultProperties(),
+        wqr.getServerProperties(InetAddress.getByName("10.119.103.112")));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.119.103.113"));
+    assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("10.221.103.121"));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.221.104.0"));
+    assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("10.222.103.121"));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.223.104.0"));
+    assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("10.113.221.221"));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.113.221.222"));
+    assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("127.0.0.1"));
+
+    TestFileBasedIPList.removeFile("fixedwhitelist.txt");
+    TestFileBasedIPList.removeFile("variablewhitelist.txt");
+  }
+
+
+  /**
+   * Add a bunch of subnets and IPSs to the whitelist
+   * Check  for inclusion in whitelist
+   * Check for exclusion from whitelist
+   */
+  public void testFixedAndLocalWhiteList() throws IOException {
+
+    String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
+
+    TestFileBasedIPList.createFileWithEntries ("fixedwhitelist.txt", fixedIps);
+
+    String[] variableIps = {"10.222.0.0/16", "10.113.221.221"};
+
+    TestFileBasedIPList.createFileWithEntries ("variablewhitelist.txt", variableIps);
+
+    Configuration conf = new Configuration();
+
+    conf.set(WhitelistBasedResolver.HADOOP_SECURITY_SASL_FIXEDWHITELIST_FILE ,
+        "fixedwhitelist.txt");
+
+    conf.setBoolean(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_ENABLE,
+        false);
+
+    conf.setLong(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_CACHE_SECS,
+        100);
+
+    conf.set(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_FILE ,
+        "variablewhitelist.txt");
+
+    WhitelistBasedResolver wqr = new WhitelistBasedResolver();
+    wqr.setConf(conf);
+
+    assertEquals (wqr.getDefaultProperties(),
+        wqr.getServerProperties(InetAddress.getByName("10.119.103.112")));
+
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.119.103.113"));
+
+    assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("10.221.103.121"));
+
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.221.104.0"));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.222.103.121"));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.223.104.0"));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.113.221.221"));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.113.221.222"));
+    assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("127.0.0.1"));;
+
+    TestFileBasedIPList.removeFile("fixedwhitelist.txt");
+    TestFileBasedIPList.removeFile("variablewhitelist.txt");
+  }
+
+  /**
+   * Add a bunch of subnets and IPSs to the whitelist
+   * Check  for inclusion in whitelist with a null value
+   */
+  public void testNullIPAddress() throws IOException {
+
+    String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
+
+    TestFileBasedIPList.createFileWithEntries ("fixedwhitelist.txt", fixedIps);
+
+    String[] variableIps = {"10.222.0.0/16", "10.113.221.221"};
+
+    TestFileBasedIPList.createFileWithEntries ("variablewhitelist.txt", variableIps);
+
+    Configuration conf = new Configuration();
+    conf.set(WhitelistBasedResolver.HADOOP_SECURITY_SASL_FIXEDWHITELIST_FILE ,
+        "fixedwhitelist.txt");
+
+    conf.setBoolean(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_ENABLE,
+        true);
+
+    conf.setLong(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_CACHE_SECS,
+        100);
+
+    conf.set(WhitelistBasedResolver.HADOOP_SECURITY_SASL_VARIABLEWHITELIST_FILE ,
+        "variablewhitelist.txt");
+
+    WhitelistBasedResolver wqr = new WhitelistBasedResolver();
+    wqr.setConf(conf);
+
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties((InetAddress)null));
+    assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties((String)null));
+
+    TestFileBasedIPList.removeFile("fixedwhitelist.txt");
+    TestFileBasedIPList.removeFile("variablewhitelist.txt");
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCacheableIPList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCacheableIPList.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCacheableIPList.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCacheableIPList.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,188 @@
+/**
+ * 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.hadoop.util;
+
+import java.io.IOException;
+
+import org.apache.hadoop.util.CacheableIPList;
+import org.apache.hadoop.util.FileBasedIPList;
+
+
+import junit.framework.TestCase;
+
+public class TestCacheableIPList extends TestCase {
+
+  /**
+   * Add a bunch of subnets and IPSs to the file
+   * setup a low cache refresh
+   * test for inclusion
+   * Check for exclusion
+   * Add a bunch of subnets and Ips
+   * wait for cache timeout.
+   * test for inclusion
+   * Check for exclusion
+   */
+  public void testAddWithSleepForCacheTimeout() throws IOException, InterruptedException {
+
+    String[] ips = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips);
+
+    CacheableIPList cipl = new CacheableIPList(
+        new FileBasedIPList("ips.txt"),100);
+
+    assertFalse("10.113.221.222 is in the list",
+        cipl.isIn("10.113.221.222"));
+    assertFalse ("10.222.103.121 is  in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+    String[]ips2 = {"10.119.103.112", "10.221.102.0/23",
+        "10.222.0.0/16", "10.113.221.221", "10.113.221.222"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips2);
+    Thread.sleep(101);
+
+    assertTrue("10.113.221.222 is not in the list",
+        cipl.isIn("10.113.221.222"));
+    assertTrue ("10.222.103.121 is not in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+  }
+
+  /**
+   * Add a bunch of subnets and IPSs to the file
+   * setup a low cache refresh
+   * test for inclusion
+   * Check for exclusion
+   * Remove a bunch of subnets and Ips
+   * wait for cache timeout.
+   * test for inclusion
+   * Check for exclusion
+   */
+  public void testRemovalWithSleepForCacheTimeout() throws IOException, InterruptedException {
+
+    String[] ips = {"10.119.103.112", "10.221.102.0/23",
+        "10.222.0.0/16", "10.113.221.221", "10.113.221.222"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips);
+
+    CacheableIPList cipl = new CacheableIPList(
+        new FileBasedIPList("ips.txt"),100);
+
+    assertTrue("10.113.221.222 is not in the list",
+        cipl.isIn("10.113.221.222"));
+    assertTrue ("10.222.103.121 is not in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+    String[]ips2 = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips2);
+    Thread.sleep(1005);
+
+    assertFalse("10.113.221.222 is in the list",
+        cipl.isIn("10.113.221.222"));
+    assertFalse ("10.222.103.121 is  in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+  }
+
+  /**
+   * Add a bunch of subnets and IPSs to the file
+   * setup a low cache refresh
+   * test for inclusion
+   * Check for exclusion
+   * Add a bunch of subnets and Ips
+   * do a refresh
+   * test for inclusion
+   * Check for exclusion
+   */
+  public void testAddWithRefresh() throws IOException, InterruptedException {
+
+    String[] ips = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips);
+
+    CacheableIPList cipl = new CacheableIPList(
+        new FileBasedIPList("ips.txt"),100);
+
+    assertFalse("10.113.221.222 is in the list",
+        cipl.isIn("10.113.221.222"));
+    assertFalse ("10.222.103.121 is  in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+    String[]ips2 = {"10.119.103.112", "10.221.102.0/23",
+        "10.222.0.0/16", "10.113.221.221", "10.113.221.222"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips2);
+    cipl.refresh();
+
+    assertTrue("10.113.221.222 is not in the list",
+        cipl.isIn("10.113.221.222"));
+    assertTrue ("10.222.103.121 is not in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+  }
+
+  /**
+   * Add a bunch of subnets and IPSs to the file
+   * setup a low cache refresh
+   * test for inclusion
+   * Check for exclusion
+   * Remove a bunch of subnets and Ips
+   * wait for cache timeout.
+   * test for inclusion
+   * Check for exclusion
+   */
+  public void testRemovalWithRefresh() throws IOException, InterruptedException {
+
+    String[] ips = {"10.119.103.112", "10.221.102.0/23",
+        "10.222.0.0/16", "10.113.221.221", "10.113.221.222"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips);
+
+    CacheableIPList cipl = new CacheableIPList(
+        new FileBasedIPList("ips.txt"),100);
+
+    assertTrue("10.113.221.222 is not in the list",
+        cipl.isIn("10.113.221.222"));
+    assertTrue ("10.222.103.121 is not in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+    String[]ips2 = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
+
+    TestFileBasedIPList.createFileWithEntries ("ips.txt", ips2);
+    cipl.refresh();
+
+    assertFalse("10.113.221.222 is in the list",
+        cipl.isIn("10.113.221.222"));
+    assertFalse ("10.222.103.121 is  in the list",
+        cipl.isIn("10.222.103.121"));
+
+    TestFileBasedIPList.removeFile("ips.txt");
+  }
+
+
+
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFileBasedIPList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFileBasedIPList.java?rev=1618484&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFileBasedIPList.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFileBasedIPList.java Sun Aug 17 17:43:20 2014
@@ -0,0 +1,215 @@
+/**
+ * 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.hadoop.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.util.FileBasedIPList;
+import org.apache.hadoop.util.IPList;
+import org.junit.After;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestFileBasedIPList extends TestCase {
+
+  @After
+  public void tearDown() {
+    removeFile("ips.txt");
+  }
+
+  /**
+   * Add a bunch of IPS  to the file
+   * Check  for inclusion
+   * Check for exclusion
+   */
+  @Test
+  public void testSubnetsAndIPs() throws IOException {
+
+    String[] ips = {"10.119.103.112", "10.221.102.0/23"};
+
+    createFileWithEntries ("ips.txt", ips);
+
+    IPList ipList = new FileBasedIPList("ips.txt");
+
+    assertTrue ("10.119.103.112 is not in the list",
+        ipList.isIn("10.119.103.112"));
+    assertFalse ("10.119.103.113 is in the list",
+        ipList.isIn("10.119.103.113"));
+
+    assertTrue ("10.221.102.0 is not in the list",
+        ipList.isIn("10.221.102.0"));
+    assertTrue ("10.221.102.1 is not in the list",
+        ipList.isIn("10.221.102.1"));
+    assertTrue ("10.221.103.1 is not in the list",
+        ipList.isIn("10.221.103.1"));
+    assertTrue ("10.221.103.255 is not in the list",
+        ipList.isIn("10.221.103.255"));
+    assertFalse("10.221.104.0 is in the list",
+        ipList.isIn("10.221.104.0"));
+    assertFalse("10.221.104.1 is in the list",
+        ipList.isIn("10.221.104.1"));
+  }
+
+  /**
+   * Add a bunch of IPS  to the file
+   * Check  for inclusion
+   * Check for exclusion
+   */
+  @Test
+  public void testNullIP() throws IOException {
+
+    String[] ips = {"10.119.103.112", "10.221.102.0/23"};
+    createFileWithEntries ("ips.txt", ips);
+
+    IPList ipList = new FileBasedIPList("ips.txt");
+
+    assertFalse ("Null Ip is in the list",
+        ipList.isIn(null));
+  }
+
+  /**
+   * Add a bunch of subnets and IPSs to the file
+   * Check  for inclusion
+   * Check for exclusion
+   */
+  @Test
+  public void testWithMultipleSubnetAndIPs() throws IOException {
+
+    String[] ips = {"10.119.103.112", "10.221.102.0/23", "10.222.0.0/16",
+    "10.113.221.221"};
+
+    createFileWithEntries ("ips.txt", ips);
+
+    IPList ipList = new FileBasedIPList("ips.txt");
+
+    assertTrue ("10.119.103.112 is not in the list",
+        ipList.isIn("10.119.103.112"));
+    assertFalse ("10.119.103.113 is in the list",
+        ipList.isIn("10.119.103.113"));
+
+    assertTrue ("10.221.103.121 is not in the list",
+        ipList.isIn("10.221.103.121"));
+    assertFalse("10.221.104.0 is in the list",
+        ipList.isIn("10.221.104.0"));
+
+    assertTrue ("10.222.103.121 is not in the list",
+        ipList.isIn("10.222.103.121"));
+    assertFalse("10.223.104.0 is in the list",
+        ipList.isIn("10.223.104.0"));
+
+    assertTrue ("10.113.221.221 is not in the list",
+        ipList.isIn("10.113.221.221"));
+    assertFalse("10.113.221.222 is in the list",
+        ipList.isIn("10.113.221.222"));
+  }
+
+  /**
+   * Do not specify the file
+   * test for inclusion
+   * should be true as if the feature is turned off
+   */
+  public void testFileNotSpecified() {
+
+    IPList ipl = new FileBasedIPList(null);
+
+    assertFalse("110.113.221.222 is in the list",
+        ipl.isIn("110.113.221.222"));
+  }
+
+  /**
+   * Specify a non existent file
+   * test for inclusion
+   * should be true as if the feature is turned off
+   */
+  public void testFileMissing() {
+
+    IPList ipl = new FileBasedIPList("missingips.txt");
+
+    assertFalse("110.113.221.222 is in the list",
+        ipl.isIn("110.113.221.222"));
+  }
+
+  /**
+   * Specify an existing file, but empty
+   * test for inclusion
+   * should be true as if the feature is turned off
+   */
+  public void testWithEmptyList() throws IOException {
+    String[] ips = {};
+
+    createFileWithEntries ("ips.txt", ips);
+    IPList ipl = new FileBasedIPList("ips.txt");
+
+    assertFalse("110.113.221.222 is in the list",
+        ipl.isIn("110.113.221.222"));
+  }
+
+  /**
+   * Specify an existing file, but ips in wrong format
+   * test for inclusion
+   * should be true as if the feature is turned off
+   */
+  public void testForBadFIle() throws IOException {
+    String[] ips = { "10.221.102/23"};
+
+    createFileWithEntries ("ips.txt", ips);
+
+    try {
+      new FileBasedIPList("ips.txt");
+     fail();
+     } catch (Exception e) {
+       //expects Exception
+     }
+  }
+
+  /**
+   * Add a bunch of subnets and IPSs to the file. Keep one entry wrong.
+   * The good entries will still be used.
+   * Check  for inclusion with good entries
+   * Check for exclusion
+   */
+  public void testWithAWrongEntry() throws IOException {
+
+    String[] ips = {"10.119.103.112", "10.221.102/23", "10.221.204.1/23"};
+
+    createFileWithEntries ("ips.txt", ips);
+
+    try {
+     new FileBasedIPList("ips.txt");
+    fail();
+    } catch (Exception e) {
+      //expects Exception
+    }
+  }
+
+  public static void createFileWithEntries(String fileName, String[] ips)
+      throws IOException {
+    FileUtils.writeLines(new File(fileName), Arrays.asList(ips));
+  }
+
+  public static void removeFile(String fileName) {
+    File file  = new File(fileName);
+    if (file.exists()) {
+      new File(fileName).delete();
+    }
+  }
+}