You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2013/11/19 06:39:49 UTC

svn commit: r1543314 [5/6] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/ hbase-common/s...

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,84 @@
+/**
+ * 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.hbase.security.visibility;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.Iterator;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.io.util.StreamUtils;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * This Filter checks the visibility expression with each KV against visibility labels associated
+ * with the scan. Based on the check the KV is included in the scan result or gets filtered out.
+ */
+@InterfaceAudience.Private
+class VisibilityLabelFilter extends FilterBase {
+
+  private BitSet authLabels;
+
+  public VisibilityLabelFilter(BitSet authLabels) {
+    this.authLabels = authLabels;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(Cell cell) throws IOException {
+    Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
+        cell.getTagsLength());
+    while (tagsItr.hasNext()) {
+      boolean includeKV = true;
+      Tag tag = tagsItr.next();
+      if (tag.getType() == VisibilityUtils.VISIBILITY_TAG_TYPE) {
+        int offset = tag.getTagOffset();
+        int endOffset = offset + tag.getTagLength();
+        while (offset < endOffset) {
+          Pair<Integer, Integer> result = StreamUtils.readRawVarint32(tag.getBuffer(), offset);
+          int currLabelOrdinal = result.getFirst();
+          if (currLabelOrdinal < 0) {
+            // check for the absence of this label in the Scan Auth labels
+            // ie. to check BitSet corresponding bit is 0
+            int temp = -currLabelOrdinal;
+            if (this.authLabels.get(temp)) {
+              includeKV = false;
+              break;
+            }
+          } else {
+            if (!this.authLabels.get(currLabelOrdinal)) {
+              includeKV = false;
+              break;
+            }
+          }
+          offset += result.getSecond();
+        }
+        if (includeKV) {
+          // We got one visibility expression getting evaluated to true. Good to include this KV in
+          // the result then.
+          return ReturnCode.INCLUDE;
+        }
+        return ReturnCode.SKIP;
+      }
+    }
+    return ReturnCode.INCLUDE;
+  }
+}
\ No newline at end of file

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsManager.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsManager.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsManager.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,183 @@
+/*
+ * 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.hbase.security.visibility;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Maintains the cache for visibility labels and also uses the zookeeper to update the labels in the
+ * system. The cache updation happens based on the data change event that happens on the zookeeper
+ * znode for labels table
+ */
+@InterfaceAudience.Private
+public class VisibilityLabelsManager {
+
+  private static final Log LOG = LogFactory.getLog(VisibilityLabelsManager.class);
+  private static final List<String> EMPTY_LIST = new ArrayList<String>(0);
+  private static VisibilityLabelsManager instance;
+
+  private ZKVisibilityLabelWatcher zkVisibilityWatcher;
+  private Map<String, Integer> labels = new HashMap<String, Integer>();
+  private Map<Integer, String> ordinalVsLabels = new HashMap<Integer, String>();
+  private Map<String, Set<Integer>> userAuths = new HashMap<String, Set<Integer>>();
+  private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+  private VisibilityLabelsManager(ZooKeeperWatcher watcher, Configuration conf) throws IOException {
+    zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf);
+    try {
+      zkVisibilityWatcher.start();
+    } catch (KeeperException ke) {
+      LOG.error("ZooKeeper initialization failed", ke);
+      throw new IOException(ke);
+    }
+  }
+
+  public synchronized static VisibilityLabelsManager get(ZooKeeperWatcher watcher,
+      Configuration conf) throws IOException {
+    if (instance == null) {
+      instance = new VisibilityLabelsManager(watcher, conf);
+    }
+    return instance;
+  }
+
+  public static VisibilityLabelsManager get() {
+    return instance;
+  }
+
+  public void refreshLabelsCache(byte[] data) throws IOException {
+    List<VisibilityLabel> visibilityLabels = null;
+    try {
+      visibilityLabels = VisibilityUtils.readLabelsFromZKData(data);
+    } catch (DeserializationException dse) {
+      throw new IOException(dse);
+    }
+    this.lock.writeLock().lock();
+    try {
+      for (VisibilityLabel visLabel : visibilityLabels) {
+        String label = Bytes.toString(visLabel.getLabel().toByteArray());
+        labels.put(label, visLabel.getOrdinal());
+        ordinalVsLabels.put(visLabel.getOrdinal(), label);
+      }
+    } finally {
+      this.lock.writeLock().unlock();
+    }
+  }
+
+  public void refreshUserAuthsCache(byte[] data) throws IOException {
+    MultiUserAuthorizations multiUserAuths = null;
+    try {
+      multiUserAuths = VisibilityUtils.readUserAuthsFromZKData(data);
+    } catch (DeserializationException dse) {
+      throw new IOException(dse);
+    }
+    this.lock.writeLock().lock();
+    try {
+      for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) {
+        String user = Bytes.toString(userAuths.getUser().toByteArray());
+        this.userAuths.put(user, new HashSet<Integer>(userAuths.getAuthList()));
+      }
+    } finally {
+      this.lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * @param label
+   * @return The ordinal for the label. The ordinal starts from 1. Returns 0 when the passed a non
+   *         existing label.
+   */
+  public int getLabelOrdinal(String label) {
+    Integer ordinal = null;
+    this.lock.readLock().lock();
+    try {
+      ordinal = labels.get(label);
+    } finally {
+      this.lock.readLock().unlock();
+    }
+    if (ordinal != null) {
+      return ordinal.intValue();
+    }
+    // 0 denotes not available
+    return 0;
+  }
+
+  public String getLabel(int ordinal) {
+    this.lock.readLock().lock();
+    try {
+      return this.ordinalVsLabels.get(ordinal);
+    } finally {
+      this.lock.readLock().unlock();
+    }
+  }
+
+  /**
+   * @return The total number of visibility labels.
+   */
+  public int getLabelsCount(){
+    return this.labels.size();
+  }
+
+  /**
+   * @param user
+   * @return The labels that the given user is authorized for.
+   */
+  public List<String> getAuths(String user) {
+    List<String> auths = EMPTY_LIST;
+    this.lock.readLock().lock();
+    try {
+      Set<Integer> authOrdinals = userAuths.get(user);
+      if (authOrdinals != null) {
+        auths = new ArrayList<String>(authOrdinals.size());
+        for (Integer authOrdinal : authOrdinals) {
+          auths.add(ordinalVsLabels.get(authOrdinal));
+        }
+      }
+    } finally {
+      this.lock.readLock().unlock();
+    }
+    return auths;
+  }
+
+  /**
+   * Writes the labels data to zookeeper node.
+   * @param data
+   * @param labelsOrUserAuths true for writing labels and false for user auths.
+   */
+  public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) {
+    this.zkVisibilityWatcher.writeToZookeeper(data, labelsOrUserAuths);
+  }
+}

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,133 @@
+/**
+ * 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.hbase.security.visibility;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
+/**
+ * Utility method to support visibility
+ */
+@InterfaceAudience.Private
+public class VisibilityUtils {
+
+  public static final String VISIBILITY_LABEL_GENERATOR_CLASS = 
+      "hbase.regionserver.scan.visibility.label.generator.class";
+  public static final byte VISIBILITY_TAG_TYPE = (byte) 2;
+  public static final String SYSTEM_LABEL = "system";
+
+  /**
+   * Creates the labels data to be written to zookeeper.
+   * @param existingLabels
+   * @return Bytes form of labels and their ordinal details to be written to zookeeper.
+   */
+  public static byte[] getDataToWriteToZooKeeper(Map<String, Integer> existingLabels) {
+    VisibilityLabelsRequest.Builder visReqBuilder = VisibilityLabelsRequest.newBuilder();
+    for (Entry<String, Integer> entry : existingLabels.entrySet()) {
+      VisibilityLabel.Builder visLabBuilder = VisibilityLabel.newBuilder();
+      visLabBuilder.setLabel(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(entry.getKey())));
+      visLabBuilder.setOrdinal(entry.getValue());
+      visReqBuilder.addVisLabel(visLabBuilder.build());
+    }
+    return ProtobufUtil.prependPBMagic(visReqBuilder.build().toByteArray());
+  }
+
+  /**
+   * Creates the user auth data to be written to zookeeper.
+   * @param userAuths
+   * @return Bytes form of user auths details to be written to zookeeper.
+   */
+  public static byte[] getUserAuthsDataToWriteToZooKeeper(Map<String, List<Integer>> userAuths) {
+    MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
+    for (Entry<String, List<Integer>> entry : userAuths.entrySet()) {
+      UserAuthorizations.Builder userAuthsBuilder = UserAuthorizations.newBuilder();
+      userAuthsBuilder.setUser(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(entry.getKey())));
+      for (Integer label : entry.getValue()) {
+        userAuthsBuilder.addAuth(label);
+      }
+      builder.addUserAuths(userAuthsBuilder.build());
+    }
+    return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+  }
+
+  /**
+   * Reads back from the zookeeper. The data read here is of the form written by
+   * writeToZooKeeper(Map<byte[], Integer> entries).
+   * 
+   * @param data
+   * @return Labels and their ordinal details
+   * @throws DeserializationException
+   */
+  public static List<VisibilityLabel> readLabelsFromZKData(byte[] data)
+      throws DeserializationException {
+    if (ProtobufUtil.isPBMagicPrefix(data)) {
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      try {
+        VisibilityLabelsRequest request = VisibilityLabelsRequest.newBuilder()
+            .mergeFrom(data, pblen, data.length - pblen).build();
+        return request.getVisLabelList();
+      } catch (InvalidProtocolBufferException e) {
+        throw new DeserializationException(e);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Reads back User auth data written to zookeeper.
+   * @param data
+   * @return User auth details
+   * @throws DeserializationException
+   */
+  public static MultiUserAuthorizations readUserAuthsFromZKData(byte[] data) 
+      throws DeserializationException {
+    if (ProtobufUtil.isPBMagicPrefix(data)) {
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      try {
+        MultiUserAuthorizations multiUserAuths = MultiUserAuthorizations.newBuilder()
+            .mergeFrom(data, pblen, data.length - pblen).build();
+        return multiUserAuths;
+      } catch (InvalidProtocolBufferException e) {
+        throw new DeserializationException(e);
+      }
+    }
+    return null;
+  }
+
+  public static ScanLabelGenerator getScanLabelGenerator(Configuration conf) {
+    Class<? extends ScanLabelGenerator> scanLabelGeneratorKlass = conf
+        .getClass(VISIBILITY_LABEL_GENERATOR_CLASS, DefaultScanLabelGenerator.class,
+            ScanLabelGenerator.class);
+    return ReflectionUtils.newInstance(scanLabelGeneratorKlass, conf);
+  }
+}

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,143 @@
+/**
+ * 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.hbase.security.visibility;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * A zk watcher that watches the labels table znode. This would create a znode
+ * /hbase/visibility_labels and will have a serialized form of a set of labels in the system.
+ */
+@InterfaceAudience.Private
+public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
+
+  private static final Log LOG = LogFactory.getLog(ZKVisibilityLabelWatcher.class);
+  private static final String VISIBILITY_LABEL_ZK_PATH = "zookeeper.znode.visibility.label.parent";
+  private static final String DEFAULT_VISIBILITY_LABEL_NODE = "visibility/labels";
+  private static final String VISIBILITY_USER_AUTHS_ZK_PATH = 
+      "zookeeper.znode.visibility.user.auths.parent";
+  private static final String DEFAULT_VISIBILITY_USER_AUTHS_NODE = "visibility/user_auths";
+
+  private VisibilityLabelsManager labelsManager;
+  private String labelZnode;
+  private String userAuthsZnode;
+
+  public ZKVisibilityLabelWatcher(ZooKeeperWatcher watcher, VisibilityLabelsManager labelsManager,
+      Configuration conf) {
+    super(watcher);
+    this.labelsManager = labelsManager;
+    String labelZnodeParent = conf.get(VISIBILITY_LABEL_ZK_PATH, DEFAULT_VISIBILITY_LABEL_NODE);
+    String userAuthsZnodeParent = conf.get(VISIBILITY_USER_AUTHS_ZK_PATH,
+        DEFAULT_VISIBILITY_USER_AUTHS_NODE);
+    this.labelZnode = ZKUtil.joinZNode(watcher.baseZNode, labelZnodeParent);
+    this.userAuthsZnode = ZKUtil.joinZNode(watcher.baseZNode, userAuthsZnodeParent);
+  }
+
+  public void start() throws KeeperException {
+    watcher.registerListener(this);
+    ZKUtil.watchAndCheckExists(watcher, labelZnode);
+    ZKUtil.watchAndCheckExists(watcher, userAuthsZnode);
+  }
+
+  private void refreshVisibilityLabelsCache(byte[] data) {
+    try {
+      this.labelsManager.refreshLabelsCache(data);
+    } catch (IOException ioe) {
+      LOG.error("Failed parsing data from labels table " + " from zk", ioe);
+    }
+  }
+
+  private void refreshUserAuthsCache(byte[] data) {
+    try {
+      this.labelsManager.refreshUserAuthsCache(data);
+    } catch (IOException ioe) {
+      LOG.error("Failed parsing data from labels table " + " from zk", ioe);
+    }
+  }
+
+  @Override
+  public void nodeCreated(String path) {
+    if (path.equals(labelZnode) || path.equals(userAuthsZnode)) {
+      try {
+        ZKUtil.watchAndCheckExists(watcher, path);
+      } catch (KeeperException ke) {
+        LOG.error("Error setting watcher on node " + path, ke);
+        // only option is to abort
+        watcher.abort("Zookeeper error obtaining label node children", ke);
+      }
+    }
+  }
+
+  @Override
+  public void nodeDeleted(String path) {
+    // There is no case of visibility labels path to get deleted.
+  }
+
+  @Override
+  public void nodeDataChanged(String path) {
+    if (path.equals(labelZnode) || path.equals(userAuthsZnode)) {
+      try {
+        watcher.sync(path);
+        byte[] data = ZKUtil.getDataAndWatch(watcher, path);
+        if (path.equals(labelZnode)) {
+          refreshVisibilityLabelsCache(data);
+        } else {
+          refreshUserAuthsCache(data);
+        }
+      } catch (KeeperException ke) {
+        LOG.error("Error reading data from zookeeper for node " + path, ke);
+        // only option is to abort
+        watcher.abort("Zookeeper error getting data for node " + path, ke);
+      }
+    }
+  }
+
+  @Override
+  public void nodeChildrenChanged(String path) {
+    // We are not dealing with child nodes under the label znode or userauths znode.
+  }
+
+  /**
+   * Write a labels mirror or user auths mirror into zookeeper
+   * 
+   * @param data
+   * @param labelsOrUserAuths true for writing labels and false for user auths.
+   */
+  public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) {
+    String znode = this.labelZnode;
+    if (!labelsOrUserAuths) {
+      znode = this.userAuthsZnode;
+    }
+    try {
+      ZKUtil.createWithParents(watcher, znode);
+      ZKUtil.updateExistingNodeData(watcher, znode, data, -1);
+    } catch (KeeperException e) {
+      LOG.error("Failed writing to " + znode, e);
+      watcher.abort("Failed writing node " + znode + " to zookeeper", e);
+    }
+  }
+}

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/ExpressionNode.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/ExpressionNode.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/ExpressionNode.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/ExpressionNode.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,27 @@
+/**
+ * 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.hbase.security.visibility.expression;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ExpressionNode {
+  boolean isSingleNode();
+
+  ExpressionNode deepClone();
+}

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/LeafExpressionNode.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/LeafExpressionNode.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/LeafExpressionNode.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/LeafExpressionNode.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,65 @@
+/**
+ * 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.hbase.security.visibility.expression;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class LeafExpressionNode implements ExpressionNode {
+  public static final LeafExpressionNode OPEN_PARAN_NODE = new LeafExpressionNode("(");
+  public static final LeafExpressionNode CLOSE_PARAN_NODE = new LeafExpressionNode(")");
+
+  private String identifier;
+
+  public LeafExpressionNode(String identifier) {
+    this.identifier = identifier;
+  }
+
+  public String getIdentifier() {
+    return this.identifier;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.identifier.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof LeafExpressionNode) {
+      LeafExpressionNode that = (LeafExpressionNode) obj;
+      return this.identifier.equals(that.identifier);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return this.identifier;
+  }
+
+  @Override
+  public boolean isSingleNode() {
+    return true;
+  }
+
+  public LeafExpressionNode deepClone() {
+    LeafExpressionNode clone = new LeafExpressionNode(this.identifier);
+    return clone;
+  }
+}

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/NonLeafExpressionNode.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/NonLeafExpressionNode.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/NonLeafExpressionNode.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/NonLeafExpressionNode.java Tue Nov 19 05:39:47 2013
@@ -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.hbase.security.visibility.expression;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class NonLeafExpressionNode implements ExpressionNode {
+  private Operator op;
+  private List<ExpressionNode> childExps = new ArrayList<ExpressionNode>(2);
+
+  public NonLeafExpressionNode() {
+
+  }
+
+  public NonLeafExpressionNode(Operator op) {
+    this.op = op;
+  }
+
+  public NonLeafExpressionNode(Operator op, List<ExpressionNode> exps) {
+    this.op = op;
+    if (op == Operator.NOT && exps.size() > 1) {
+      throw new IllegalArgumentException(Operator.NOT + " should be on 1 child expression");
+    }
+    this.childExps = exps;
+  }
+
+  public NonLeafExpressionNode(Operator op, ExpressionNode... exps) {
+    this.op = op;
+    List<ExpressionNode> expLst = new ArrayList<ExpressionNode>();
+    for (ExpressionNode exp : exps) {
+      expLst.add(exp);
+    }
+    this.childExps = expLst;
+  }
+
+  public Operator getOperator() {
+    return op;
+  }
+
+  public List<ExpressionNode> getChildExps() {
+    return childExps;
+  }
+
+  public void addChildExp(ExpressionNode exp) {
+    if (op == Operator.NOT && this.childExps.size() == 1) {
+      throw new IllegalStateException(Operator.NOT + " should be on 1 child expression");
+    }
+    this.childExps.add(exp);
+  }
+
+  public void addChildExps(List<ExpressionNode> exps) {
+    this.childExps.addAll(exps);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("(");
+    if (this.op == Operator.NOT) {
+      sb.append(this.op);
+    }
+    for (int i = 0; i < this.childExps.size(); i++) {
+      sb.append(childExps.get(i));
+      if (i < this.childExps.size() - 1) {
+        sb.append(" " + this.op + " ");
+      }
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  @Override
+  public boolean isSingleNode() {
+    return this.op == Operator.NOT;
+  }
+
+  public NonLeafExpressionNode deepClone() {
+    NonLeafExpressionNode clone = new NonLeafExpressionNode(this.op);
+    for (ExpressionNode exp : this.childExps) {
+      clone.addChildExp(exp.deepClone());
+    }
+    return clone;
+  }
+}

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/Operator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/Operator.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/Operator.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/Operator.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,32 @@
+/**
+ * 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.hbase.security.visibility.expression;
+
+public enum Operator {
+  AND('&'), OR('|'), NOT('!');
+
+  private char rep;
+
+  private Operator(char rep) {
+    this.rep = rep;
+  }
+
+  public String toString() {
+    return String.valueOf(this.rep);
+  };
+}

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,393 @@
+/**
+ * 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.hbase.security.visibility;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
+import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
+import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
+import org.apache.hadoop.hbase.security.visibility.expression.Operator;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestExpressionExpander {
+
+  @Test
+  public void testPositiveCases() throws Exception {
+    ExpressionExpander expander = new ExpressionExpander();
+
+    // (!a) -> (!a)
+    NonLeafExpressionNode exp1 = new NonLeafExpressionNode(Operator.NOT,
+        new LeafExpressionNode("a"));
+    ExpressionNode result = expander.expand(exp1);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    NonLeafExpressionNode nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.NOT, nlResult.getOperator());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+
+    // (a | b) -> (a | b)
+    NonLeafExpressionNode exp2 = new NonLeafExpressionNode(Operator.OR,
+        new LeafExpressionNode("a"), new LeafExpressionNode("b"));
+    result = expander.expand(exp2);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(2, nlResult.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+
+    // (a & b) -> (a & b)
+    NonLeafExpressionNode exp3 = new NonLeafExpressionNode(Operator.AND,
+        new LeafExpressionNode("a"), new LeafExpressionNode("b"));
+    result = expander.expand(exp3);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.AND, nlResult.getOperator());
+    assertEquals(2, nlResult.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+
+    // ((a | b) | c) -> (a | b | c)
+    NonLeafExpressionNode exp4 = new NonLeafExpressionNode(Operator.OR, new NonLeafExpressionNode(
+        Operator.OR, new LeafExpressionNode("a"), new LeafExpressionNode("b")),
+        new LeafExpressionNode("c"));
+    result = expander.expand(exp4);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(3, nlResult.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) nlResult.getChildExps().get(2)).getIdentifier());
+
+    // ((a & b) & c) -> (a & b & c)
+    NonLeafExpressionNode exp5 = new NonLeafExpressionNode(Operator.AND, new NonLeafExpressionNode(
+        Operator.AND, new LeafExpressionNode("a"), new LeafExpressionNode("b")),
+        new LeafExpressionNode("c"));
+    result = expander.expand(exp5);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.AND, nlResult.getOperator());
+    assertEquals(3, nlResult.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) nlResult.getChildExps().get(2)).getIdentifier());
+
+    // (a | b) & c -> ((a & c) | (b & c))
+    NonLeafExpressionNode exp6 = new NonLeafExpressionNode(Operator.AND, new NonLeafExpressionNode(
+        Operator.OR, new LeafExpressionNode("a"), new LeafExpressionNode("b")),
+        new LeafExpressionNode("c"));
+    result = expander.expand(exp6);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(2, nlResult.getChildExps().size());
+    NonLeafExpressionNode temp = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(1);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    // (a & b) | c -> ((a & b) | c)
+    NonLeafExpressionNode exp7 = new NonLeafExpressionNode(Operator.OR, new NonLeafExpressionNode(
+        Operator.AND, new LeafExpressionNode("a"), new LeafExpressionNode("b")),
+        new LeafExpressionNode("c"));
+    result = expander.expand(exp7);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(2, nlResult.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+    nlResult = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.AND, nlResult.getOperator());
+    assertEquals(2, nlResult.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+
+    // ((a & b) | c) & d -> (((a & b) & d) | (c & d))
+    NonLeafExpressionNode exp8 = new NonLeafExpressionNode(Operator.AND);
+    exp8.addChildExp(new NonLeafExpressionNode(Operator.OR, new NonLeafExpressionNode(Operator.AND,
+        new LeafExpressionNode("a"), new LeafExpressionNode("b")), new LeafExpressionNode("c")));
+    exp8.addChildExp(new LeafExpressionNode("d"));
+    result = expander.expand(exp8);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(2, nlResult.getChildExps().size());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(1);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    // (a | b) | (c | d) -> (a | b | c | d)
+    NonLeafExpressionNode exp9 = new NonLeafExpressionNode(Operator.OR);
+    exp9.addChildExp(new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("a"),
+        new LeafExpressionNode("b")));
+    exp9.addChildExp(new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("c"),
+        new LeafExpressionNode("d")));
+    result = expander.expand(exp9);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(4, nlResult.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) nlResult.getChildExps().get(2)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) nlResult.getChildExps().get(3)).getIdentifier());
+
+    // (a & b) & (c & d) -> (a & b & c & d)
+    NonLeafExpressionNode exp10 = new NonLeafExpressionNode(Operator.AND);
+    exp10.addChildExp(new NonLeafExpressionNode(Operator.AND, new LeafExpressionNode("a"),
+        new LeafExpressionNode("b")));
+    exp10.addChildExp(new NonLeafExpressionNode(Operator.AND, new LeafExpressionNode("c"),
+        new LeafExpressionNode("d")));
+    result = expander.expand(exp10);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.AND, nlResult.getOperator());
+    assertEquals(4, nlResult.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlResult.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlResult.getChildExps().get(1)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) nlResult.getChildExps().get(2)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) nlResult.getChildExps().get(3)).getIdentifier());
+
+    // (a | b) & (c | d) -> ((a & c) | (a & d) | (b & c) | (b & d))
+    NonLeafExpressionNode exp11 = new NonLeafExpressionNode(Operator.AND);
+    exp11.addChildExp(new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("a"),
+        new LeafExpressionNode("b")));
+    exp11.addChildExp(new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("c"),
+        new LeafExpressionNode("d")));
+    result = expander.expand(exp11);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(4, nlResult.getChildExps().size());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(1);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(2);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(3);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    // (((a | b) | c) | d) & e -> ((a & e) | (b & e) | (c & e) | (d & e))
+    NonLeafExpressionNode exp12 = new NonLeafExpressionNode(Operator.AND);
+    NonLeafExpressionNode tempExp1 = new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode(
+        "a"), new LeafExpressionNode("b"));
+    NonLeafExpressionNode tempExp2 = new NonLeafExpressionNode(Operator.OR, tempExp1,
+        new LeafExpressionNode("c"));
+    NonLeafExpressionNode tempExp3 = new NonLeafExpressionNode(Operator.OR, tempExp2,
+        new LeafExpressionNode("d"));
+    exp12.addChildExp(tempExp3);
+    exp12.addChildExp(new LeafExpressionNode("e"));
+    result = expander.expand(exp12);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(4, nlResult.getChildExps().size());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(1);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(2);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(3);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    // (a | b | c) & d -> ((a & d) | (b & d) | (c & d))
+    NonLeafExpressionNode exp13 = new NonLeafExpressionNode(Operator.AND,
+        new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("a"), new LeafExpressionNode(
+            "b"), new LeafExpressionNode("c")), new LeafExpressionNode("d"));
+    result = expander.expand(exp13);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(3, nlResult.getChildExps().size());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(1);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(2);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    // ((a | b) & (c | d)) & (e | f) -> (((a & c) & e) | ((a & c) & f) | ((a & d) & e) | ((a & d) &
+    // f) | ((b & c) & e) | ((b & c) & f) | ((b & d) & e) | ((b & d) & f))
+    NonLeafExpressionNode exp15 = new NonLeafExpressionNode(Operator.AND);
+    NonLeafExpressionNode temp1 = new NonLeafExpressionNode(Operator.AND);
+    temp1.addChildExp(new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("a"),
+        new LeafExpressionNode("b")));
+    temp1.addChildExp(new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("c"),
+        new LeafExpressionNode("d")));
+    exp15.addChildExp(temp1);
+    exp15.addChildExp(new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("e"),
+        new LeafExpressionNode("f")));
+    result = expander.expand(exp15);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.OR, nlResult.getOperator());
+    assertEquals(8, nlResult.getChildExps().size());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(1);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("f", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(2);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(3);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("f", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(4);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(5);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("f", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("c", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(6);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("e", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(7);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("f", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+    temp = (NonLeafExpressionNode) temp.getChildExps().get(0);
+    assertEquals(Operator.AND, temp.getOperator());
+    assertEquals(2, temp.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) temp.getChildExps().get(1)).getIdentifier());
+
+    // !(a | b) -> ((!a) & (!b))
+    NonLeafExpressionNode exp16 = new NonLeafExpressionNode(Operator.NOT,
+        new NonLeafExpressionNode(Operator.OR, new LeafExpressionNode("a"), new LeafExpressionNode(
+            "b")));
+    result = expander.expand(exp16);
+    assertTrue(result instanceof NonLeafExpressionNode);
+    nlResult = (NonLeafExpressionNode) result;
+    assertEquals(Operator.AND, nlResult.getOperator());
+    assertEquals(2, nlResult.getChildExps().size());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(0);
+    assertEquals(Operator.NOT, temp.getOperator());
+    assertEquals("a", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+    temp = (NonLeafExpressionNode) nlResult.getChildExps().get(1);
+    assertEquals(Operator.NOT, temp.getOperator());
+    assertEquals("b", ((LeafExpressionNode) temp.getChildExps().get(0)).getIdentifier());
+  }
+}

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,318 @@
+/**
+ * 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.hbase.security.visibility;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
+import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
+import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
+import org.apache.hadoop.hbase.security.visibility.expression.Operator;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestExpressionParser {
+
+  private ExpressionParser parser = new ExpressionParser();
+
+  @Test
+  public void testPositiveCases() throws Exception {
+    // abc -> (abc)
+    ExpressionNode node = parser.parse("abc");
+    assertTrue(node instanceof LeafExpressionNode);
+    assertEquals("abc", ((LeafExpressionNode) node).getIdentifier());
+
+    // a&b|c&d -> (((a & b) | c) & )
+    node = parser.parse("a&b|c&d");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("d", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.OR, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+
+    // (a) -> (a)
+    node = parser.parse("(a)");
+    assertTrue(node instanceof LeafExpressionNode);
+    assertEquals("a", ((LeafExpressionNode) node).getIdentifier());
+
+    // (a&b) -> (a & b)
+    node = parser.parse(" ( a & b )");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+
+    // ((((a&b)))) -> (a & b)
+    node = parser.parse("((((a&b))))");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+
+    // (a|b)&(cc|def) -> ((a | b) & (cc | def))
+    node = parser.parse("( a | b ) & (cc|def)");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    NonLeafExpressionNode nlNodeLeft = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    NonLeafExpressionNode nlNodeRight = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.OR, nlNodeLeft.getOperator());
+    assertEquals(2, nlNodeLeft.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNodeLeft.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlNodeLeft.getChildExps().get(1)).getIdentifier());
+    assertEquals(Operator.OR, nlNodeRight.getOperator());
+    assertEquals(2, nlNodeRight.getChildExps().size());
+    assertEquals("cc", ((LeafExpressionNode) nlNodeRight.getChildExps().get(0)).getIdentifier());
+    assertEquals("def", ((LeafExpressionNode) nlNodeRight.getChildExps().get(1)).getIdentifier());
+
+    // a&(cc|de) -> (a & (cc | de))
+    node = parser.parse("a&(cc|de)");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.OR, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("cc", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertEquals("de", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+
+    // (a&b)|c -> ((a & b) | c)
+    node = parser.parse("(a&b)|c");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.OR, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+
+    // (a&b&c)|d -> (((a & b) & c) | d)
+    node = parser.parse("(a&b&c)|d");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.OR, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("d", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+
+    // a&(b|(c|d)) -> (a & (b | (c | d)))
+    node = parser.parse("a&(b|(c|d))");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.OR, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.OR, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertEquals("d", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+
+    // (!a) -> (!a)
+    node = parser.parse("(!a)");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.NOT, nlNode.getOperator());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+
+    // a&(!b) -> (a & (!b))
+    node = parser.parse("a&(!b)");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.NOT, nlNode.getOperator());
+    assertEquals(1, nlNode.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+
+    // !a&b -> ((!a) & b)
+    node = parser.parse("!a&b");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.NOT, nlNode.getOperator());
+    assertEquals(1, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+
+    // !a&(!b) -> ((!a) & (!b))
+    node = parser.parse("!a&(!b)");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNodeLeft = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    nlNodeRight = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.NOT, nlNodeLeft.getOperator());
+    assertEquals(1, nlNodeLeft.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNodeLeft.getChildExps().get(0)).getIdentifier());
+    assertEquals(Operator.NOT, nlNodeRight.getOperator());
+    assertEquals(1, nlNodeRight.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNodeRight.getChildExps().get(0)).getIdentifier());
+
+    // !a&!b -> ((!a) & (!b))
+    node = parser.parse("!a&!b");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNodeLeft = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    nlNodeRight = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.NOT, nlNodeLeft.getOperator());
+    assertEquals(1, nlNodeLeft.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNodeLeft.getChildExps().get(0)).getIdentifier());
+    assertEquals(Operator.NOT, nlNodeRight.getOperator());
+    assertEquals(1, nlNodeRight.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNodeRight.getChildExps().get(0)).getIdentifier());
+
+    // !(a&b) -> (!(a & b))
+    node = parser.parse("!(a&b)");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.NOT, nlNode.getOperator());
+    assertEquals(1, nlNode.getChildExps().size());
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(1)).getIdentifier());
+
+    // a&!b -> (a & (!b))
+    node = parser.parse("a&!b");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertEquals("a", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.NOT, nlNode.getOperator());
+    assertEquals(1, nlNode.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNode.getChildExps().get(0)).getIdentifier());
+
+    // !((a|b)&!(c&!b)) -> (!((a | b) & (!(c & (!b)))))
+    node = parser.parse("!((a | b) & !(c & !b))");
+    assertTrue(node instanceof NonLeafExpressionNode);
+    nlNode = (NonLeafExpressionNode) node;
+    assertEquals(Operator.NOT, nlNode.getOperator());
+    assertEquals(1, nlNode.getChildExps().size());
+    nlNode = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    assertEquals(Operator.AND, nlNode.getOperator());
+    assertEquals(2, nlNode.getChildExps().size());
+    assertTrue(nlNode.getChildExps().get(0) instanceof NonLeafExpressionNode);
+    assertTrue(nlNode.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNodeLeft = (NonLeafExpressionNode) nlNode.getChildExps().get(0);
+    nlNodeRight = (NonLeafExpressionNode) nlNode.getChildExps().get(1);
+    assertEquals(Operator.OR, nlNodeLeft.getOperator());
+    assertEquals("a", ((LeafExpressionNode) nlNodeLeft.getChildExps().get(0)).getIdentifier());
+    assertEquals("b", ((LeafExpressionNode) nlNodeLeft.getChildExps().get(1)).getIdentifier());
+    assertEquals(Operator.NOT, nlNodeRight.getOperator());
+    assertEquals(1, nlNodeRight.getChildExps().size());
+    nlNodeRight = (NonLeafExpressionNode) nlNodeRight.getChildExps().get(0);
+    assertEquals(Operator.AND, nlNodeRight.getOperator());
+    assertEquals(2, nlNodeRight.getChildExps().size());
+    assertEquals("c", ((LeafExpressionNode) nlNodeRight.getChildExps().get(0)).getIdentifier());
+    assertTrue(nlNodeRight.getChildExps().get(1) instanceof NonLeafExpressionNode);
+    nlNodeRight = (NonLeafExpressionNode) nlNodeRight.getChildExps().get(1);
+    assertEquals(Operator.NOT, nlNodeRight.getOperator());
+    assertEquals(1, nlNodeRight.getChildExps().size());
+    assertEquals("b", ((LeafExpressionNode) nlNodeRight.getChildExps().get(0)).getIdentifier());
+  }
+
+  @Test
+  public void testNegativeCases() throws Exception {
+    executeNegativeCase("(");
+    executeNegativeCase(")");
+    executeNegativeCase("()");
+    executeNegativeCase("(a");
+    executeNegativeCase("a&");
+    executeNegativeCase("a&|b");
+    executeNegativeCase("!");
+    executeNegativeCase("a!");
+    executeNegativeCase("a!&");
+    executeNegativeCase("&");
+    executeNegativeCase("|");
+    executeNegativeCase("!(a|(b&c)&!b");
+    executeNegativeCase("!!a");
+    executeNegativeCase("( a & b ) | ( c & d e)");
+    executeNegativeCase("! a");
+  }
+
+  private void executeNegativeCase(String exp) {
+    try {
+      parser.parse(exp);
+      fail("Expected ParseException for expression " + exp);
+    } catch (ParseException e) {
+    }
+  }
+}