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 [1/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...

Author: anoopsamjohn
Date: Tue Nov 19 05:39:47 2013
New Revision: 1543314

URL: http://svn.apache.org/r1543314
Log:
HBASE-7663 [Per-KV security] Visibility labels

Added:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityConstants.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsValidator.java
    hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
    hbase/trunk/hbase-protocol/src/main/protobuf/VisibilityLabels.proto
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultScanLabelGenerator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ExpressionExpander.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ExpressionParser.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ParseException.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ScanLabelGenerator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/SimpleScanLabelGenerator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/ExpressionNode.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/LeafExpressionNode.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/NonLeafExpressionNode.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/expression/Operator.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java
    hbase/trunk/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
    hbase/trunk/hbase-shell/src/main/ruby/shell/commands/add_labels.rb
    hbase/trunk/hbase-shell/src/main/ruby/shell/commands/clear_auths.rb
    hbase/trunk/hbase-shell/src/main/ruby/shell/commands/get_auths.rb
    hbase/trunk/hbase-shell/src/main/ruby/shell/commands/set_auths.rb
Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
    hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
    hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto
    hbase/trunk/hbase-shell/src/main/ruby/hbase.rb
    hbase/trunk/hbase-shell/src/main/ruby/hbase/hbase.rb
    hbase/trunk/hbase-shell/src/main/ruby/shell.rb
    hbase/trunk/hbase-shell/src/main/ruby/shell/commands.rb

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java?rev=1543314&r1=1543313&r2=1543314&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java Tue Nov 19 05:39:47 2013
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.util.Byte
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class Get extends OperationWithAttributes
+public class Get extends Query
   implements Row, Comparable<Row> {
 
   private byte [] row = null;

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java?rev=1543314&r1=1543313&r2=1543314&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java Tue Nov 19 05:39:47 2013
@@ -36,7 +36,11 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.security.visibility.CellVisibility;
+import org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 
@@ -290,6 +294,26 @@ public abstract class Mutation extends O
   }
 
   /**
+   * Sets the visibility expression associated with cells in this Mutation.
+   * It is illegal to set <code>CellVisibility</code> on <code>Delete</code> mutation.
+   * @param expression
+   */
+  public void setCellVisibility(CellVisibility expression) {
+    this.setAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY, ProtobufUtil
+        .toCellVisibility(expression).toByteArray());
+  }
+
+  /**
+   * @return CellVisibility associated with cells in this Mutation.
+   * @throws DeserializationException
+   */
+  public CellVisibility getCellVisibility() throws DeserializationException {
+    byte[] cellVisibilityBytes = this.getAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY);
+    if (cellVisibilityBytes == null) return null;
+    return ProtobufUtil.toCellVisibility(cellVisibilityBytes);
+  }
+
+  /**
    * Number of KeyValues carried by this Mutation.
    * @return the total number of KeyValues
    */

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,49 @@
+/**
+ * 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.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.security.visibility.Authorizations;
+import org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class Query extends OperationWithAttributes {
+
+  /**
+   * Sets the authorizations to be used by this Query
+   * @param authorizations
+   */
+  public void setAuthorizations(Authorizations authorizations) {
+    this.setAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY, ProtobufUtil
+        .toAuthorizations(authorizations).toByteArray());
+  }
+
+  /**
+   * @return The authorizations this Query is associated with.
+   * @throws DeserializationException
+   */
+  public Authorizations getAuthorizations() throws DeserializationException {
+    byte[] authorizationsBytes = this.getAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY);
+    if (authorizationsBytes == null) return null;
+    return ProtobufUtil.toAuthorizations(authorizationsBytes);
+  }
+}

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java?rev=1543314&r1=1543313&r2=1543314&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java Tue Nov 19 05:39:47 2013
@@ -81,7 +81,7 @@ import java.util.TreeSet;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class Scan extends OperationWithAttributes {
+public class Scan extends Query {
   private static final String RAW_ATTR = "_raw_";
   private static final String ISOLATION_LEVEL = "_isolationlevel_";
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1543314&r1=1543313&r2=1543314&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Tue Nov 19 05:39:47 2013
@@ -119,6 +119,8 @@ import org.apache.hadoop.hbase.security.
 import org.apache.hadoop.hbase.security.access.TablePermission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
+import org.apache.hadoop.hbase.security.visibility.Authorizations;
+import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.Methods;
@@ -2464,4 +2466,89 @@ public final class ProtobufUtil {
     return tableNames;
   }
 
+  /**
+   * Convert a protocol buffer CellVisibility to a client CellVisibility
+   * 
+   * @param proto
+   * @return the converted client CellVisibility
+   */
+  public static CellVisibility toCellVisibility(ClientProtos.CellVisibility proto) {
+    if (proto == null) return null;
+    return new CellVisibility(proto.getExpression());
+  }
+
+  /**
+   * Convert a protocol buffer CellVisibility bytes to a client CellVisibility
+   * 
+   * @param protoBytes
+   * @return the converted client CellVisibility
+   * @throws DeserializationException
+   */
+  public static CellVisibility toCellVisibility(byte[] protoBytes) throws DeserializationException {
+    if (protoBytes == null) return null;
+    ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
+    ClientProtos.CellVisibility proto = null;
+    try {
+      proto = builder.mergeFrom(protoBytes).build();
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return toCellVisibility(proto);
+  }
+
+  /**
+   * Create a protocol buffer CellVisibility based on a client CellVisibility.
+   * 
+   * @param cellVisibility
+   * @return a protocol buffer CellVisibility
+   */
+  public static ClientProtos.CellVisibility toCellVisibility(CellVisibility cellVisibility) {
+    ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
+    builder.setExpression(cellVisibility.getExpression());
+    return builder.build();
+  }
+
+  /**
+   * Convert a protocol buffer Authorizations to a client Authorizations
+   * 
+   * @param proto
+   * @return the converted client Authorizations
+   */
+  public static Authorizations toAuthorizations(ClientProtos.Authorizations proto) {
+    if (proto == null) return null;
+    return new Authorizations(proto.getLabelList());
+  }
+
+  /**
+   * Convert a protocol buffer Authorizations bytes to a client Authorizations
+   * 
+   * @param protoBytes
+   * @return the converted client Authorizations
+   * @throws DeserializationException
+   */
+  public static Authorizations toAuthorizations(byte[] protoBytes) throws DeserializationException {
+    if (protoBytes == null) return null;
+    ClientProtos.Authorizations.Builder builder = ClientProtos.Authorizations.newBuilder();
+    ClientProtos.Authorizations proto = null;
+    try {
+      proto = builder.mergeFrom(protoBytes).build();
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return toAuthorizations(proto);
+  }
+
+  /**
+   * Create a protocol buffer Authorizations based on a client Authorizations.
+   * 
+   * @param authorizations
+   * @return a protocol buffer Authorizations
+   */
+  public static ClientProtos.Authorizations toAuthorizations(Authorizations authorizations) {
+    ClientProtos.Authorizations.Builder builder = ClientProtos.Authorizations.newBuilder();
+    for (String label : authorizations.getLabels()) {
+      builder.addLabel(label);
+    }
+    return builder.build();
+  }
 }

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,56 @@
+/**
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This class contains visibility labels associated with a Scan/Get deciding which all labeled data
+ * current scan/get can access.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class Authorizations {
+
+  private List<String> labels;
+
+  public Authorizations(String... labels) {
+    this.labels = new ArrayList<String>(labels.length);
+    for (String label : labels) {
+      this.labels.add(label);
+    }
+  }
+
+  public Authorizations(List<String> labels) {
+    this.labels = labels;
+  }
+
+  public List<String> getLabels() {
+    return Collections.unmodifiableList(this.labels);
+  }
+  
+  @Override
+  public String toString() {
+    return this.labels.toString();
+  }
+}
\ No newline at end of file

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,44 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This contains a visibility expression which can be associated with a cell. When it is set with a
+ * Mutation, all the cells in that mutation will get associated with this expression. A visibility
+ * expression can contain visibility labels combined with logical operators AND(&), OR(|) and NOT(!)
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CellVisibility {
+
+  private String expression;
+
+  public CellVisibility(String expression) {
+    this.expression = expression;
+  }
+
+  /**
+   * @return The visibility expression
+   */
+  public String getExpression() {
+    return this.expression;
+  }
+}

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java Tue Nov 19 05:39:47 2013
@@ -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.hbase.security.visibility;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InvalidLabelException extends DoNotRetryIOException {
+  private static final long serialVersionUID = 1L;
+
+  public InvalidLabelException(String msg) {
+    super(msg);
+  }
+}
+

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java Tue Nov 19 05:39:47 2013
@@ -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.hbase.security.visibility;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class LabelAlreadyExistsException extends DoNotRetryIOException {
+  private static final long serialVersionUID = 1L;
+
+  public LabelAlreadyExistsException(String msg) {
+    super(msg);
+  }
+
+}

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,207 @@
+/**
+ * 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.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
+/**
+ * Utility client for doing visibility labels admin operations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class VisibilityClient {
+
+  /**
+   * Utility method for adding label to the system.
+   * 
+   * @param conf
+   * @param label
+   * @return VisibilityLabelsResponse
+   * @throws Throwable
+   */
+  public static VisibilityLabelsResponse addLabel(Configuration conf, final String label)
+      throws Throwable {
+    return addLabels(conf, new String[] { label });
+  }
+
+  /**
+   * Utility method for adding labels to the system.
+   * 
+   * @param conf
+   * @param labels
+   * @return VisibilityLabelsResponse
+   * @throws Throwable
+   */
+  public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
+      throws Throwable {
+    HTable ht = null;
+    try {
+      ht = new HTable(conf, LABELS_TABLE_NAME.getName());
+      Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable = 
+          new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
+        ServerRpcController controller = new ServerRpcController();
+        BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback = 
+            new BlockingRpcCallback<VisibilityLabelsResponse>();
+
+        public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
+          VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
+          for (String label : labels) {
+            if (label.length() > 0) {
+              VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
+              newBuilder.setLabel(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(label)));
+              builder.addVisLabel(newBuilder.build());
+            }
+          }
+          service.addLabels(controller, builder.build(), rpcCallback);
+          return rpcCallback.get();
+        }
+      };
+      Map<byte[], VisibilityLabelsResponse> result = ht.coprocessorService(
+          VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
+          callable);
+      return result.values().iterator().next(); // There will be exactly one region for labels
+                                                // table and so one entry in result Map.
+    } finally {
+      if (ht != null) {
+        ht.close();
+      }
+    }
+  }
+
+  /**
+   * Sets given labels globally authorized for the user.
+   * @param conf
+   * @param auths
+   * @param user
+   * @return VisibilityLabelsResponse
+   * @throws Throwable
+   */
+  public static VisibilityLabelsResponse setAuths(Configuration conf, final String[] auths,
+      final String user) throws Throwable {
+    return setOrClearAuths(conf, auths, user, true);
+  }
+
+  /**
+   * @param conf
+   * @param user
+   * @return labels, the given user is globally authorized for.
+   * @throws Throwable
+   */
+  public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
+    HTable ht = null;
+    try {
+      ht = new HTable(conf, LABELS_TABLE_NAME.getName());
+      Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable = 
+          new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
+        ServerRpcController controller = new ServerRpcController();
+        BlockingRpcCallback<GetAuthsResponse> rpcCallback = 
+            new BlockingRpcCallback<GetAuthsResponse>();
+
+        public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
+          GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
+          getAuthReqBuilder.setUser(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(user)));
+          service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
+          return rpcCallback.get();
+        }
+      };
+      Map<byte[], GetAuthsResponse> result = ht.coprocessorService(VisibilityLabelsService.class,
+          HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, callable);
+      return result.values().iterator().next(); // There will be exactly one region for labels
+                                                // table and so one entry in result Map.
+    } finally {
+      if (ht != null) {
+        ht.close();
+      }
+    }
+  }
+
+  /**
+   * Removes given labels from user's globally authorized list of labels.
+   * @param conf
+   * @param auths
+   * @param user
+   * @return VisibilityLabelsResponse
+   * @throws Throwable
+   */
+  public static VisibilityLabelsResponse clearAuths(Configuration conf, final String[] auths,
+      final String user) throws Throwable {
+    return setOrClearAuths(conf, auths, user, false);
+  }
+
+  private static VisibilityLabelsResponse setOrClearAuths(Configuration conf, final String[] auths,
+      final String user, final boolean setOrClear) throws IOException, ServiceException, Throwable {
+    HTable ht = null;
+    try {
+      ht = new HTable(conf, LABELS_TABLE_NAME.getName());
+      Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable = 
+          new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
+        ServerRpcController controller = new ServerRpcController();
+        BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback = 
+            new BlockingRpcCallback<VisibilityLabelsResponse>();
+
+        public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
+          SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
+          setAuthReqBuilder.setUser(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(user)));
+          for (String auth : auths) {
+            if (auth.length() > 0) {
+              setAuthReqBuilder.addAuth(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(auth)));
+            }
+          }
+          if (setOrClear) {
+            service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
+          } else {
+            service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
+          }
+          return rpcCallback.get();
+        }
+      };
+      Map<byte[], VisibilityLabelsResponse> result = ht.coprocessorService(
+          VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
+          callable);
+      return result.values().iterator().next(); // There will be exactly one region for labels
+                                                // table and so one entry in result Map.
+    } finally {
+      if (ht != null) {
+        ht.close();
+      }
+    }
+  }
+}

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityConstants.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityConstants.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityConstants.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,43 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+
+@InterfaceAudience.Private
+public final class VisibilityConstants {
+
+  /**
+   * The string that is used as key in setting the Operation attributes for visibility labels
+   */
+  public static final String VISIBILITY_LABELS_ATTR_KEY = "VISIBILITY";
+
+  /** Internal storage table for visibility labels */
+  public static final TableName LABELS_TABLE_NAME = TableName.valueOf(
+      NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "labels");
+
+  /** Family for the internal storage table for visibility labels */
+  public static final byte[] LABELS_TABLE_FAMILY = Bytes.toBytes("f");
+
+  /** Qualifier for the internal storage table for visibility labels */
+  public static final byte[] LABEL_QUALIFIER = new byte[1];
+
+}

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsValidator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsValidator.java?rev=1543314&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsValidator.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsValidator.java Tue Nov 19 05:39:47 2013
@@ -0,0 +1,66 @@
+/*
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A simple validator that validates the labels passed
+ */
+@InterfaceAudience.Private
+public class VisibilityLabelsValidator {
+  // We follow Accumulo parity for valid visibility labels.
+  private static final boolean[] validAuthChars = new boolean[256];
+
+  static {
+    for (int i = 0; i < 256; i++) {
+      validAuthChars[i] = false;
+    }
+
+    for (int i = 'a'; i <= 'z'; i++) {
+      validAuthChars[i] = true;
+    }
+
+    for (int i = 'A'; i <= 'Z'; i++) {
+      validAuthChars[i] = true;
+    }
+
+    for (int i = '0'; i <= '9'; i++) {
+      validAuthChars[i] = true;
+    }
+
+    validAuthChars['_'] = true;
+    validAuthChars['-'] = true;
+    validAuthChars[':'] = true;
+    validAuthChars['.'] = true;
+    validAuthChars['/'] = true;
+  }
+  
+  static final boolean isValidAuthChar(byte b) {
+    return validAuthChars[0xff & b];
+  }
+
+  static final boolean isValidLabel(byte[] label) {
+    for (int i = 0; i < label.length; i++) {
+      if (!isValidAuthChar(label[i])) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java?rev=1543314&r1=1543313&r2=1543314&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java Tue Nov 19 05:39:47 2013
@@ -24,6 +24,7 @@ import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.base.Preconditions;
 
@@ -38,7 +39,6 @@ import com.google.common.base.Preconditi
 public class StreamUtils {
 
   public static void writeRawVInt32(OutputStream output, int value) throws IOException {
-    assert value >= 0;
     while (true) {
       if ((value & ~0x7F) == 0) {
         output.write(value);
@@ -118,6 +118,57 @@ public class StreamUtils {
     return result;
   }
 
+  /**
+   * Reads a varInt value stored in an array.
+   * 
+   * @param input
+   *          Input array where the varInt is available
+   * @param offset
+   *          Offset in the input array where varInt is available
+   * @return A pair of integers in which first value is the actual decoded varInt value and second
+   *         value as number of bytes taken by this varInt for it's storage in the input array.
+   * @throws IOException
+   */
+  public static Pair<Integer, Integer> readRawVarint32(byte[] input, int offset) throws IOException {
+    int newOffset = offset;
+    byte tmp = input[newOffset++];
+    if (tmp >= 0) {
+      return new Pair<Integer, Integer>((int) tmp, newOffset - offset);
+    }
+    int result = tmp & 0x7f;
+    tmp = input[newOffset++];
+    if (tmp >= 0) {
+      result |= tmp << 7;
+    } else {
+      result |= (tmp & 0x7f) << 7;
+      tmp = input[newOffset++];
+      if (tmp >= 0) {
+        result |= tmp << 14;
+      } else {
+        result |= (tmp & 0x7f) << 14;
+        tmp = input[newOffset++];
+        if (tmp >= 0) {
+          result |= tmp << 21;
+        } else {
+          result |= (tmp & 0x7f) << 21;
+          tmp = input[newOffset++];
+          result |= tmp << 28;
+          if (tmp < 0) {
+            // Discard upper 32 bits.
+            for (int i = 0; i < 5; i++) {
+              tmp = input[newOffset++];
+              if (tmp >= 0) {
+                return new Pair<Integer, Integer>(result, newOffset - offset);
+              }
+            }
+            throw new IOException("Malformed varint");
+          }
+        }
+      }
+    }
+    return new Pair<Integer, Integer>(result, newOffset - offset);
+  }
+
   public static short toShort(byte hi, byte lo) {
     short s = (short) (((hi & 0xFF) << 8) | (lo & 0xFF));
     Preconditions.checkArgument(s >= 0);