View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   * http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.security.visibility;
20  
21  import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SANITY_CHECK_FAILURE;
22  import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SUCCESS;
23  import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_FAMILY;
24  import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
25  import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABEL_QUALIFIER;
26  import static org.apache.hadoop.hbase.security.visibility.VisibilityUtils.SYSTEM_LABEL;
27  
28  import java.io.ByteArrayOutputStream;
29  import java.io.DataOutputStream;
30  import java.io.IOException;
31  import java.util.ArrayList;
32  import java.util.BitSet;
33  import java.util.HashMap;
34  import java.util.Iterator;
35  import java.util.List;
36  import java.util.Map;
37  
38  import com.google.protobuf.HBaseZeroCopyByteString;
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.classification.InterfaceAudience;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.Cell;
44  import org.apache.hadoop.hbase.CellScanner;
45  import org.apache.hadoop.hbase.CellUtil;
46  import org.apache.hadoop.hbase.CoprocessorEnvironment;
47  import org.apache.hadoop.hbase.DoNotRetryIOException;
48  import org.apache.hadoop.hbase.HColumnDescriptor;
49  import org.apache.hadoop.hbase.HConstants;
50  import org.apache.hadoop.hbase.HRegionInfo;
51  import org.apache.hadoop.hbase.HTableDescriptor;
52  import org.apache.hadoop.hbase.KeyValue;
53  import org.apache.hadoop.hbase.KeyValue.Type;
54  import org.apache.hadoop.hbase.KeyValueUtil;
55  import org.apache.hadoop.hbase.NamespaceDescriptor;
56  import org.apache.hadoop.hbase.ServerName;
57  import org.apache.hadoop.hbase.TableName;
58  import org.apache.hadoop.hbase.Tag;
59  import org.apache.hadoop.hbase.catalog.MetaReader;
60  import org.apache.hadoop.hbase.client.Delete;
61  import org.apache.hadoop.hbase.client.Get;
62  import org.apache.hadoop.hbase.client.Mutation;
63  import org.apache.hadoop.hbase.client.Put;
64  import org.apache.hadoop.hbase.client.Result;
65  import org.apache.hadoop.hbase.client.Scan;
66  import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
67  import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
68  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
69  import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
70  import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
71  import org.apache.hadoop.hbase.coprocessor.MasterObserver;
72  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
73  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
74  import org.apache.hadoop.hbase.coprocessor.RegionObserver;
75  import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
76  import org.apache.hadoop.hbase.exceptions.DeserializationException;
77  import org.apache.hadoop.hbase.filter.Filter;
78  import org.apache.hadoop.hbase.filter.FilterList;
79  import org.apache.hadoop.hbase.io.hfile.HFile;
80  import org.apache.hadoop.hbase.io.util.StreamUtils;
81  import org.apache.hadoop.hbase.ipc.RequestContext;
82  import org.apache.hadoop.hbase.master.MasterServices;
83  import org.apache.hadoop.hbase.master.RegionPlan;
84  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
85  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
86  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
87  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos;
88  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
89  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
90  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
91  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
92  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
93  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
94  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
95  import org.apache.hadoop.hbase.regionserver.BloomType;
96  import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
97  import org.apache.hadoop.hbase.regionserver.HRegion;
98  import org.apache.hadoop.hbase.regionserver.InternalScanner;
99  import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
100 import org.apache.hadoop.hbase.regionserver.OperationStatus;
101 import org.apache.hadoop.hbase.regionserver.RegionScanner;
102 import org.apache.hadoop.hbase.security.AccessDeniedException;
103 import org.apache.hadoop.hbase.security.User;
104 import org.apache.hadoop.hbase.security.access.AccessControlLists;
105 import org.apache.hadoop.hbase.security.access.AccessController;
106 import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
107 import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
108 import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
109 import org.apache.hadoop.hbase.security.visibility.expression.Operator;
110 import org.apache.hadoop.hbase.util.Bytes;
111 import org.apache.hadoop.hbase.util.Pair;
112 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
113 
114 import com.google.common.collect.Lists;
115 import com.google.common.collect.MapMaker;
116 import com.google.protobuf.ByteString;
117 import com.google.protobuf.RpcCallback;
118 import com.google.protobuf.RpcController;
119 import com.google.protobuf.Service;
120 
121 /**
122  * Coprocessor that has both the MasterObserver and RegionObserver implemented that supports in
123  * visibility labels
124  */
125 @InterfaceAudience.Private
126 public class VisibilityController extends BaseRegionObserver implements MasterObserver,
127     RegionObserver, VisibilityLabelsService.Interface, CoprocessorService {
128 
129   private static final Log LOG = LogFactory.getLog(VisibilityController.class);
130   private static final byte[] DUMMY_VALUE = new byte[0];
131   // "system" label is having an ordinal value 1.
132   private static final int SYSTEM_LABEL_ORDINAL = 1;
133   private static final Tag[] LABELS_TABLE_TAGS = new Tag[1];
134 
135   private final ExpressionParser expressionParser = new ExpressionParser();
136   private final ExpressionExpander expressionExpander = new ExpressionExpander();
137   private VisibilityLabelsManager visibilityManager;
138   // defined only for Endpoint implementation, so it can have way to access region services.
139   private RegionCoprocessorEnvironment regionEnv;
140   private ScanLabelGenerator scanLabelGenerator;
141   
142   private volatile int ordinalCounter = -1;
143   // flags if we are running on a region of the 'labels' table
144   private boolean labelsRegion = false;
145   // Flag denoting whether AcessController is available or not.
146   private boolean acOn = false;
147   private Configuration conf;
148   private volatile boolean initialized = false;
149 
150   /** Mapping of scanner instances to the user who created them */
151   private Map<InternalScanner,String> scannerOwners =
152       new MapMaker().weakKeys().makeMap();
153 
154   static {
155     ByteArrayOutputStream baos = new ByteArrayOutputStream();
156     DataOutputStream dos = new DataOutputStream(baos);
157     try {
158       StreamUtils.writeRawVInt32(dos, SYSTEM_LABEL_ORDINAL);
159     } catch (IOException e) {
160       // We write to a byte array. No Exception can happen.
161     }
162     LABELS_TABLE_TAGS[0] = new Tag(VisibilityUtils.VISIBILITY_TAG_TYPE, baos.toByteArray());
163   }
164 
165   @Override
166   public void start(CoprocessorEnvironment env) throws IOException {
167     this.conf = env.getConfiguration();
168     if (HFile.getFormatVersion(conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
169       throw new RuntimeException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
170         + " is required to persist visibility labels. Consider setting " + HFile.FORMAT_VERSION_KEY
171         + " accordingly.");
172     }
173     ZooKeeperWatcher zk = null;
174     if (env instanceof MasterCoprocessorEnvironment) {
175       // if running on HMaster
176       MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment) env;
177       zk = mEnv.getMasterServices().getZooKeeper();
178     } else if (env instanceof RegionCoprocessorEnvironment) {
179       // if running at region
180       regionEnv = (RegionCoprocessorEnvironment) env;
181       zk = regionEnv.getRegionServerServices().getZooKeeper();
182     } else if (env instanceof RegionServerCoprocessorEnvironment) {
183       throw new RuntimeException(
184           "Visibility controller should not be configured as " +
185           "'hbase.coprocessor.regionserver.classes'.");
186     }
187 
188     // If zk is null or IOException while obtaining auth manager,
189     // throw RuntimeException so that the coprocessor is unloaded.
190     if (zk == null) {
191       throw new RuntimeException("Error obtaining VisibilityLabelsManager, zk found null.");
192     }
193     try {
194       this.visibilityManager = VisibilityLabelsManager.get(zk, this.conf);
195     } catch (IOException ioe) {
196       throw new RuntimeException("Error obtaining VisibilityLabelsManager", ioe);
197     }
198     if (env instanceof RegionCoprocessorEnvironment) {
199       // ScanLabelGenerator to be instantiated only with Region Observer.
200       scanLabelGenerator = VisibilityUtils.getScanLabelGenerator(this.conf);
201     }
202   }
203 
204   @Override
205   public void stop(CoprocessorEnvironment env) throws IOException {
206 
207   }
208 
209   /********************************* Master related hooks **********************************/
210 
211   @Override
212   public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
213     // Need to create the new system table for labels here
214     MasterServices master = ctx.getEnvironment().getMasterServices();
215     if (!MetaReader.tableExists(master.getCatalogTracker(), LABELS_TABLE_NAME)) {
216       HTableDescriptor labelsTable = new HTableDescriptor(LABELS_TABLE_NAME);
217       HColumnDescriptor labelsColumn = new HColumnDescriptor(LABELS_TABLE_FAMILY);
218       labelsColumn.setBloomFilterType(BloomType.NONE);
219       labelsColumn.setBlockCacheEnabled(false); // We will cache all the labels. No need of normal
220                                                  // table block cache.
221       labelsTable.addFamily(labelsColumn);
222       // Let the "labels" table having only one region always. We are not expecting too many labels in
223       // the system.
224       labelsTable.setValue(HTableDescriptor.SPLIT_POLICY,
225           DisabledRegionSplitPolicy.class.getName());
226       labelsTable.setValue(Bytes.toBytes(HConstants.DISALLOW_WRITES_IN_RECOVERING),
227           Bytes.toBytes(true));
228       master.createTable(labelsTable, null);
229     }
230   }
231 
232   @Override
233   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
234       HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
235   }
236 
237   @Override
238   public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
239       HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
240   }
241 
242   @Override
243   public void preCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
244       HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
245   }
246 
247   @Override
248   public void postCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
249       HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
250   }
251 
252   @Override
253   public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
254       throws IOException {
255   }
256 
257   @Override
258   public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
259       throws IOException {
260   }
261 
262   @Override
263   public void preDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
264       TableName tableName) throws IOException {
265   }
266 
267   @Override
268   public void postDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
269       TableName tableName) throws IOException {
270   }
271 
272   @Override
273   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
274       TableName tableName, HTableDescriptor htd) throws IOException {
275   }
276 
277   @Override
278   public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
279       TableName tableName, HTableDescriptor htd) throws IOException {
280   }
281 
282   @Override
283   public void preModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
284       TableName tableName, HTableDescriptor htd) throws IOException {
285   }
286 
287   @Override
288   public void postModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
289       TableName tableName, HTableDescriptor htd) throws IOException {
290   }
291 
292   @Override
293   public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
294       HColumnDescriptor column) throws IOException {
295   }
296 
297   @Override
298   public void postAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
299       HColumnDescriptor column) throws IOException {
300   }
301 
302   @Override
303   public void preAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
304       TableName tableName, HColumnDescriptor column) throws IOException {
305   }
306 
307   @Override
308   public void postAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
309       TableName tableName, HColumnDescriptor column) throws IOException {
310   }
311 
312   @Override
313   public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
314       TableName tableName, HColumnDescriptor descriptor) throws IOException {
315   }
316 
317   @Override
318   public void postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
319       TableName tableName, HColumnDescriptor descriptor) throws IOException {
320   }
321 
322   @Override
323   public void preModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
324       TableName tableName, HColumnDescriptor descriptor) throws IOException {
325   }
326 
327   @Override
328   public void postModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
329       TableName tableName, HColumnDescriptor descriptor) throws IOException {
330   }
331 
332   @Override
333   public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
334       TableName tableName, byte[] c) throws IOException {
335   }
336 
337   @Override
338   public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
339       TableName tableName, byte[] c) throws IOException {
340   }
341 
342   @Override
343   public void preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
344       TableName tableName, byte[] c) throws IOException {
345   }
346 
347   @Override
348   public void postDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
349       TableName tableName, byte[] c) throws IOException {
350   }
351 
352   @Override
353   public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
354       throws IOException {
355   }
356 
357   @Override
358   public void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
359       throws IOException {
360   }
361 
362   @Override
363   public void preEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
364       TableName tableName) throws IOException {
365   }
366 
367   @Override
368   public void postEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
369       TableName tableName) throws IOException {
370   }
371 
372   @Override
373   public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
374       throws IOException {
375   }
376 
377   @Override
378   public void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
379       TableName tableName) throws IOException {
380   }
381 
382   @Override
383   public void preDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
384       TableName tableName) throws IOException {
385   }
386 
387   @Override
388   public void postDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
389       TableName tableName) throws IOException {
390   }
391 
392   @Override
393   public void preMove(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo region,
394       ServerName srcServer, ServerName destServer) throws IOException {
395   }
396 
397   @Override
398   public void postMove(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo region,
399       ServerName srcServer, ServerName destServer) throws IOException {
400   }
401 
402   @Override
403   public void preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo)
404       throws IOException {
405   }
406 
407   @Override
408   public void postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo)
409       throws IOException {
410   }
411 
412   @Override
413   public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
414       HRegionInfo regionInfo, boolean force) throws IOException {
415   }
416 
417   @Override
418   public void postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
419       HRegionInfo regionInfo, boolean force) throws IOException {
420   }
421 
422   @Override
423   public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
424       HRegionInfo regionInfo) throws IOException {
425   }
426 
427   @Override
428   public void postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
429       HRegionInfo regionInfo) throws IOException {
430   }
431 
432   @Override
433   public void preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
434   }
435 
436   @Override
437   public void postBalance(ObserverContext<MasterCoprocessorEnvironment> ctx, List<RegionPlan> plans)
438       throws IOException {
439   }
440 
441   @Override
442   public boolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
443       boolean newValue) throws IOException {
444     return false;
445   }
446 
447   @Override
448   public void postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
449       boolean oldValue, boolean newValue) throws IOException {
450   }
451 
452   @Override
453   public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
454   }
455 
456   @Override
457   public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
458   }
459 
460   @Override
461   public void preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
462       SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
463   }
464 
465   @Override
466   public void postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
467       SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
468   }
469 
470   @Override
471   public void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
472       SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
473   }
474 
475   @Override
476   public void postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
477       SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
478   }
479 
480   @Override
481   public void preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
482       SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
483   }
484 
485   @Override
486   public void postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
487       SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
488   }
489 
490   @Override
491   public void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
492       SnapshotDescription snapshot) throws IOException {
493   }
494 
495   @Override
496   public void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
497       SnapshotDescription snapshot) throws IOException {
498   }
499 
500   @Override
501   public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
502       List<TableName> tableNamesList, List<HTableDescriptor> descriptors) throws IOException {
503   }
504 
505   @Override
506   public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
507       List<HTableDescriptor> descriptors) throws IOException {
508   }
509 
510   @Override
511   public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
512       NamespaceDescriptor ns) throws IOException {
513   }
514 
515   @Override
516   public void postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
517       NamespaceDescriptor ns) throws IOException {
518   }
519 
520   @Override
521   public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, 
522       String namespace) throws IOException {
523   }
524 
525   @Override
526   public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
527       String namespace) throws IOException {
528   }
529 
530   @Override
531   public void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
532       NamespaceDescriptor ns) throws IOException {
533   }
534 
535   @Override
536   public void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
537       NamespaceDescriptor ns) throws IOException {
538   }
539 
540   @Override
541   public void preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx)
542       throws IOException {
543 
544   }
545 
546   /****************************** Region related hooks ******************************/
547 
548   @Override
549   public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
550     // Read the entire labels table and populate the zk
551     if (e.getEnvironment().getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
552       this.labelsRegion = true;
553       this.acOn = CoprocessorHost.getLoadedCoprocessors().contains(AccessController.class.getName());
554       if (!e.getEnvironment().getRegion().isRecovering()) {
555         initialize(e);
556       }
557     } else {
558       this.initialized = true;
559     }
560   }
561 
562   @Override
563   public void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e) {
564     if (this.labelsRegion) {
565       initialize(e);
566     }
567   }
568 
569   private void initialize(ObserverContext<RegionCoprocessorEnvironment> e) {
570     try {
571       Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths = 
572           extractLabelsAndAuths(getExistingLabelsWithAuths());
573       Map<String, Integer> labels = labelsAndUserAuths.getFirst();
574       Map<String, List<Integer>> userAuths = labelsAndUserAuths.getSecond();
575       // Add the "system" label if it is not added into the system yet
576       addSystemLabel(e.getEnvironment().getRegion(), labels, userAuths);
577       int ordinal = 1; // Ordinal 1 is reserved for "system" label.
578       for (Integer i : labels.values()) {
579         if (i > ordinal) {
580           ordinal = i;
581         }
582       }
583       this.ordinalCounter = ordinal + 1;
584       if (labels.size() > 0) {
585         // If there is no data need not write to zk
586         byte[] serialized = VisibilityUtils.getDataToWriteToZooKeeper(labels);
587         this.visibilityManager.writeToZookeeper(serialized, true);
588       }
589       if (userAuths.size() > 0) {
590         byte[] serialized = VisibilityUtils.getUserAuthsDataToWriteToZooKeeper(userAuths);
591         this.visibilityManager.writeToZookeeper(serialized, false);
592       }
593       initialized = true;
594     } catch (IOException ioe) {
595       LOG.error("Error while updating the zk with the exisiting labels data", ioe);
596     }
597   }
598 
599   private void addSystemLabel(HRegion region, Map<String, Integer> labels,
600       Map<String, List<Integer>> userAuths) throws IOException {
601     if (!labels.containsKey(SYSTEM_LABEL)) {
602       Put p = new Put(Bytes.toBytes(SYSTEM_LABEL_ORDINAL));
603       p.addImmutable(LABELS_TABLE_FAMILY, LABEL_QUALIFIER, Bytes.toBytes(SYSTEM_LABEL));
604       // Set auth for "system" label for all super users.
605       List<String> superUsers = getSystemAndSuperUsers();
606       for (String superUser : superUsers) {
607         p.addImmutable(
608             LABELS_TABLE_FAMILY, Bytes.toBytes(superUser), DUMMY_VALUE, LABELS_TABLE_TAGS);
609       }
610       region.put(p);
611       labels.put(SYSTEM_LABEL, SYSTEM_LABEL_ORDINAL);
612       for (String superUser : superUsers) {
613         List<Integer> auths = userAuths.get(superUser);
614         if (auths == null) {
615           auths = new ArrayList<Integer>(1);
616           userAuths.put(superUser, auths);
617         }
618         auths.add(SYSTEM_LABEL_ORDINAL);
619       }
620     }
621   }
622 
623   @Override
624   public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
625       MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
626     if (c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) {
627       return;
628     }
629     // TODO this can be made as a global LRU cache at HRS level?
630     Map<String, List<Tag>> labelCache = new HashMap<String, List<Tag>>();
631     for (int i = 0; i < miniBatchOp.size(); i++) {
632       Mutation m = miniBatchOp.getOperation(i);
633       CellVisibility cellVisibility = null;
634       try {
635         cellVisibility = m.getCellVisibility();
636       } catch (DeserializationException de) {
637         miniBatchOp.setOperationStatus(i,
638             new OperationStatus(SANITY_CHECK_FAILURE, de.getMessage()));
639         continue;
640       }
641       if (m instanceof Put) {
642         Put p = (Put) m;
643         boolean sanityFailure = false;
644         for (CellScanner cellScanner = p.cellScanner(); cellScanner.advance();) {
645           if (!checkForReservedVisibilityTagPresence(cellScanner.current())) {
646             miniBatchOp.setOperationStatus(i, new OperationStatus(SANITY_CHECK_FAILURE,
647                 "Mutation contains cell with reserved type tag"));
648             sanityFailure = true;
649             break;
650           }
651         }
652         if (!sanityFailure) {
653           if (cellVisibility != null) {
654             String labelsExp = cellVisibility.getExpression();
655             List<Tag> visibilityTags = labelCache.get(labelsExp);
656             if (visibilityTags == null) {
657               try {
658                 visibilityTags = createVisibilityTags(labelsExp);
659               } catch (ParseException e) {
660                 miniBatchOp.setOperationStatus(i,
661                     new OperationStatus(SANITY_CHECK_FAILURE, e.getMessage()));
662               } catch (InvalidLabelException e) {
663                 miniBatchOp.setOperationStatus(i,
664                     new OperationStatus(SANITY_CHECK_FAILURE, e.getMessage()));
665               }
666             }
667             if (visibilityTags != null) {
668               labelCache.put(labelsExp, visibilityTags);
669               List<Cell> updatedCells = new ArrayList<Cell>();
670               for (CellScanner cellScanner = p.cellScanner(); cellScanner.advance();) {
671                 Cell cell = cellScanner.current();
672                 List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
673                     cell.getTagsLength());
674                 tags.addAll(visibilityTags);
675                 Cell updatedCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
676                     cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(),
677                     cell.getFamilyLength(), cell.getQualifierArray(), cell.getQualifierOffset(),
678                     cell.getQualifierLength(), cell.getTimestamp(), Type.codeToType(cell
679                         .getTypeByte()), cell.getValueArray(), cell.getValueOffset(),
680                     cell.getValueLength(), tags);
681                 updatedCells.add(updatedCell);
682               }
683               p.getFamilyCellMap().clear();
684               // Clear and add new Cells to the Mutation.
685               for (Cell cell : updatedCells) {
686                 p.add(cell);
687               }
688             }
689           }
690         }
691       } else if (cellVisibility != null) {
692         // CellVisibility in a Delete is not legal! Fail the operation
693         miniBatchOp.setOperationStatus(i, new OperationStatus(SANITY_CHECK_FAILURE,
694             "CellVisibility cannot be set on Delete mutation"));
695       }
696     }
697   }
698 
699   @Override
700   public void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
701       MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
702     if (this.labelsRegion) {
703       // We will add to zookeeper here.
704       Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths =
705           extractLabelsAndAuths(getExistingLabelsWithAuths());
706       Map<String, Integer> existingLabels = labelsAndUserAuths.getFirst();
707       Map<String, List<Integer>> userAuths = labelsAndUserAuths.getSecond();
708       boolean isNewLabels = false;
709       boolean isUserAuthsChange = false;
710       for (int i = 0; i < miniBatchOp.size(); i++) {
711         Mutation m = miniBatchOp.getOperation(i);
712         if (miniBatchOp.getOperationStatus(i).getOperationStatusCode() == SUCCESS) {
713           for (List<Cell> cells : m.getFamilyCellMap().values()) {
714             for (Cell cell : cells) {
715               int labelOrdinal = Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
716               if (Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
717                       cell.getQualifierLength(), LABEL_QUALIFIER, 0,
718                       LABEL_QUALIFIER.length)) {
719                 if (m instanceof Put) {
720                   existingLabels.put(
721                       Bytes.toString(cell.getValueArray(), cell.getValueOffset(),
722                           cell.getValueLength()), labelOrdinal);
723                   isNewLabels = true;
724                 }
725               } else {
726                 String user = Bytes.toString(cell.getQualifierArray(),
727                     cell.getQualifierOffset(), cell.getQualifierLength());
728                 List<Integer> auths = userAuths.get(user);
729                 if (auths == null) {
730                   auths = new ArrayList<Integer>();
731                   userAuths.put(user, auths);
732                 }
733                 if (m instanceof Delete) {
734                   auths.remove(Integer.valueOf(labelOrdinal));
735                 } else {
736                   auths.add(labelOrdinal);
737                 }
738                 isUserAuthsChange = true;
739               }
740             }
741           }
742         }
743       }
744       if (isNewLabels) {
745         byte[] serialized = VisibilityUtils.getDataToWriteToZooKeeper(existingLabels);
746         this.visibilityManager.writeToZookeeper(serialized, true);
747       }
748       if (isUserAuthsChange) {
749         byte[] serialized = VisibilityUtils.getUserAuthsDataToWriteToZooKeeper(userAuths);
750         this.visibilityManager.writeToZookeeper(serialized, false);
751       }
752     }
753   }
754 
755   private Pair<Map<String, Integer>, Map<String, List<Integer>>> extractLabelsAndAuths(
756       List<List<Cell>> labelDetails) {
757     Map<String, Integer> labels = new HashMap<String, Integer>();
758     Map<String, List<Integer>> userAuths = new HashMap<String, List<Integer>>();
759     for (List<Cell> cells : labelDetails) {
760       for (Cell cell : cells) {
761         if (Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
762             cell.getQualifierLength(), LABEL_QUALIFIER, 0, LABEL_QUALIFIER.length)) {
763           labels.put(
764               Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()),
765               Bytes.toInt(cell.getRowArray(), cell.getRowOffset()));
766         } else {
767           // These are user cells who has authorization for this label
768           String user = Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(),
769               cell.getQualifierLength());
770           List<Integer> auths = userAuths.get(user);
771           if (auths == null) {
772             auths = new ArrayList<Integer>();
773             userAuths.put(user, auths);
774           }
775           auths.add(Bytes.toInt(cell.getRowArray(), cell.getRowOffset()));
776         }
777       }
778     }
779     return new Pair<Map<String, Integer>, Map<String, List<Integer>>>(labels, userAuths);
780   }
781 
782   // Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE.
783   // This tag type is reserved and should not be explicitly set by user.
784   private boolean checkForReservedVisibilityTagPresence(Cell cell) throws IOException {
785     if (cell.getTagsLength() > 0) {
786       Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
787           cell.getTagsLength());
788       while (tagsItr.hasNext()) {
789         if (tagsItr.next().getType() == VisibilityUtils.VISIBILITY_TAG_TYPE) {
790           return false;
791         }
792       }
793     }
794     return true;
795   }
796 
797   private List<Tag> createVisibilityTags(String visibilityLabelsExp) throws IOException,
798       ParseException, InvalidLabelException {
799     ExpressionNode node = null;
800     node = this.expressionParser.parse(visibilityLabelsExp);
801     node = this.expressionExpander.expand(node);
802     List<Tag> tags = new ArrayList<Tag>();
803     ByteArrayOutputStream baos = new ByteArrayOutputStream();
804     DataOutputStream dos = new DataOutputStream(baos);
805     if (node.isSingleNode()) {
806       writeLabelOrdinalsToStream(node, dos);
807       tags.add(new Tag(VisibilityUtils.VISIBILITY_TAG_TYPE, baos.toByteArray()));
808       baos.reset();
809     } else {
810       NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
811       if (nlNode.getOperator() == Operator.OR) {
812         for (ExpressionNode child : nlNode.getChildExps()) {
813           writeLabelOrdinalsToStream(child, dos);
814           tags.add(new Tag(VisibilityUtils.VISIBILITY_TAG_TYPE, baos.toByteArray()));
815           baos.reset();
816         }
817       } else {
818         writeLabelOrdinalsToStream(nlNode, dos);
819         tags.add(new Tag(VisibilityUtils.VISIBILITY_TAG_TYPE, baos.toByteArray()));
820         baos.reset();
821       }
822     }
823     return tags;
824   }
825 
826   private void writeLabelOrdinalsToStream(ExpressionNode node, DataOutputStream dos)
827       throws IOException, InvalidLabelException {
828     if (node.isSingleNode()) {
829       String identifier = null;
830       int labelOrdinal = 0;
831       if (node instanceof LeafExpressionNode) {
832         identifier = ((LeafExpressionNode) node)
833             .getIdentifier();
834         if (LOG.isTraceEnabled()) {
835           LOG.trace("The identifier is "+identifier);
836         }
837         labelOrdinal = this.visibilityManager.getLabelOrdinal(identifier);
838       } else {
839         // This is a NOT node.
840         LeafExpressionNode lNode = (LeafExpressionNode) ((NonLeafExpressionNode) node)
841             .getChildExps().get(0);
842         identifier = lNode.getIdentifier();
843         labelOrdinal = this.visibilityManager.getLabelOrdinal(identifier);
844         labelOrdinal = -1 * labelOrdinal; // Store NOT node as -ve ordinal.
845       }
846       if (labelOrdinal == 0) {
847         throw new InvalidLabelException("Invalid visibility label " + identifier);
848       }
849       StreamUtils.writeRawVInt32(dos, labelOrdinal);
850     } else {
851       List<ExpressionNode> childExps = ((NonLeafExpressionNode) node).getChildExps();
852       for (ExpressionNode child : childExps) {
853         writeLabelOrdinalsToStream(child, dos);
854       }
855     }
856   }
857   
858   @Override
859   public RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan,
860       RegionScanner s) throws IOException {
861     HRegion region = e.getEnvironment().getRegion();
862     Authorizations authorizations = null;
863     // If a super user issues a scan, he should be able to scan the cells
864     // irrespective of the Visibility labels
865     if (checkIfScanOrGetFromSuperUser()) {
866       return s;
867     }
868     try {
869       authorizations = scan.getAuthorizations();
870     } catch (DeserializationException de) {
871       throw new IOException(de);
872     }
873     Filter visibilityLabelFilter = createVisibilityLabelFilter(region, authorizations);
874     if (visibilityLabelFilter != null) {
875       Filter filter = scan.getFilter();
876       if (filter != null) {
877         scan.setFilter(new FilterList(filter, visibilityLabelFilter));
878       } else {
879         scan.setFilter(visibilityLabelFilter);
880       }
881     }
882     return s;
883   }
884 
885   private boolean checkIfScanOrGetFromSuperUser() throws IOException {
886     User user = getActiveUser();
887     if (user != null && user.getShortName() != null) {
888       List<String> auths = this.visibilityManager.getAuths(user.getShortName());
889       return (auths.contains(SYSTEM_LABEL));
890     }
891     return false;
892   }
893 
894   @Override
895   public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
896       final Scan scan, final RegionScanner s) throws IOException {
897     User user = getActiveUser();
898     if (user != null && user.getShortName() != null) {
899       scannerOwners.put(s, user.getShortName());
900     }
901     return s;
902   }
903 
904   @Override
905   public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
906       final InternalScanner s, final List<Result> result, final int limit, final boolean hasNext)
907       throws IOException {
908     requireScannerOwner(s);
909     return hasNext;
910   }
911 
912   @Override
913   public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
914       final InternalScanner s) throws IOException {
915     requireScannerOwner(s);
916   }
917 
918   @Override
919   public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
920       final InternalScanner s) throws IOException {
921     // clean up any associated owner mapping
922     scannerOwners.remove(s);
923   }
924 
925   /**
926    * Verify, when servicing an RPC, that the caller is the scanner owner. If so, we assume that
927    * access control is correctly enforced based on the checks performed in preScannerOpen()
928    */
929   private void requireScannerOwner(InternalScanner s) throws AccessDeniedException {
930     if (RequestContext.isInRequestContext()) {
931       String requestUName = RequestContext.getRequestUserName();
932       String owner = scannerOwners.get(s);
933       if (owner != null && !owner.equals(requestUName)) {
934         throw new AccessDeniedException("User '" + requestUName + "' is not the scanner owner!");
935       }
936     }
937   }
938 
939   @Override
940   public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
941       throws IOException {
942     Authorizations authorizations = null;
943     // If a super user issues a get, he should be able to scan the cells
944     // irrespective of the Visibility labels
945     if (checkIfScanOrGetFromSuperUser()) {
946       return;
947     }
948     try {
949       authorizations = get.getAuthorizations();
950     } catch (DeserializationException de) {
951       throw new IOException(de);
952     }
953     Filter visibilityLabelFilter = createVisibilityLabelFilter(e.getEnvironment().getRegion(),
954         authorizations);
955     if (visibilityLabelFilter != null) {
956       Filter filter = get.getFilter();
957       if (filter != null) {
958         get.setFilter(new FilterList(filter, visibilityLabelFilter));
959       } else {
960         get.setFilter(visibilityLabelFilter);
961       }
962     }
963   }
964 
965   private Filter createVisibilityLabelFilter(HRegion region, Authorizations authorizations) {
966     if (authorizations == null) {
967       // No Authorizations present for this scan/Get!
968       // In case of "labels" table and user tables, create an empty auth set. In other system tables
969       // just scan with out visibility check and filtering. Checking visibility labels for META and
970       // NAMESPACE table is not needed.
971       TableName table = region.getRegionInfo().getTable();
972       if (table.isSystemTable() && !table.equals(LABELS_TABLE_NAME)) {
973         return null;
974       }
975       return new VisibilityLabelFilter(new BitSet(0));
976     }
977     Filter visibilityLabelFilter = null;
978     if (this.scanLabelGenerator != null) {
979       List<String> labels = null;
980       try {
981         labels = this.scanLabelGenerator.getLabels(getActiveUser(), authorizations);
982       } catch (Throwable t) {
983         LOG.error(t);
984       }
985       int labelsCount = this.visibilityManager.getLabelsCount();
986       BitSet bs = new BitSet(labelsCount + 1); // ordinal is index 1 based
987       if (labels != null) {
988         for (String label : labels) {
989           int labelOrdinal = this.visibilityManager.getLabelOrdinal(label);
990           if (labelOrdinal != 0) {
991             bs.set(labelOrdinal);
992           }
993         }
994       }
995       visibilityLabelFilter = new VisibilityLabelFilter(bs);
996     }
997     return visibilityLabelFilter;
998   }
999 
1000   private User getActiveUser() throws IOException {
1001     User user = RequestContext.getRequestUser();
1002     if (!RequestContext.isInRequestContext()) {
1003       // for non-rpc handling, fallback to system user
1004       user = User.getCurrent();
1005     }
1006     if (LOG.isTraceEnabled()) {
1007       LOG.trace("Current active user name is "+user.getShortName());
1008     }
1009     return user;
1010   }
1011 
1012   private List<String> getSystemAndSuperUsers() throws IOException {
1013     User user = User.getCurrent();
1014     if (user == null) {
1015       throw new IOException("Unable to obtain the current user, "
1016           + "authorization checks for internal operations will not work correctly!");
1017     }
1018     if (LOG.isTraceEnabled()) {
1019       LOG.trace("Current user name is "+user.getShortName());
1020     }
1021     String currentUser = user.getShortName();
1022     List<String> superUsers = Lists.asList(currentUser,
1023         this.conf.getStrings(AccessControlLists.SUPERUSER_CONF_KEY, new String[0]));
1024     return superUsers;
1025   }
1026 
1027   private boolean isSystemOrSuperUser() throws IOException {
1028     List<String> superUsers = getSystemAndSuperUsers();
1029     User activeUser = getActiveUser();
1030     return superUsers.contains(activeUser.getShortName());
1031   }
1032 
1033   @Override
1034   public Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
1035       MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
1036     List<Tag> tags = Lists.newArrayList();
1037     CellVisibility cellVisibility = null;
1038     try {
1039       cellVisibility = mutation.getCellVisibility();
1040     } catch (DeserializationException e) {
1041       throw new IOException(e);
1042     }
1043     if (cellVisibility == null) {
1044       return newCell;
1045     }
1046     // Adding all other tags
1047     Iterator<Tag> tagsItr = CellUtil.tagsIterator(newCell.getTagsArray(), newCell.getTagsOffset(),
1048         newCell.getTagsLength());
1049     while (tagsItr.hasNext()) {
1050       Tag tag = tagsItr.next();
1051       if (tag.getType() != VisibilityUtils.VISIBILITY_TAG_TYPE) {
1052         tags.add(tag);
1053       }
1054     }
1055     try {
1056       tags.addAll(createVisibilityTags(cellVisibility.getExpression()));
1057     } catch (ParseException e) {
1058       throw new IOException(e);
1059     }
1060 
1061     // We need to create another KV, unfortunately, because the current new KV
1062     // has no space for tags
1063     KeyValue newKv = KeyValueUtil.ensureKeyValue(newCell);
1064     byte[] bytes = newKv.getBuffer();
1065     KeyValue rewriteKv = new KeyValue(bytes, newKv.getRowOffset(), newKv.getRowLength(), bytes,
1066         newKv.getFamilyOffset(), newKv.getFamilyLength(), bytes, newKv.getQualifierOffset(),
1067         newKv.getQualifierLength(), newKv.getTimestamp(), KeyValue.Type.codeToType(newKv
1068             .getTypeByte()), bytes, newKv.getValueOffset(), newKv.getValueLength(), tags);
1069     // Preserve mvcc data
1070     rewriteKv.setMvccVersion(newKv.getMvccVersion());
1071     return rewriteKv;
1072   }
1073 
1074   @Override
1075   public Service getService() {
1076     return VisibilityLabelsProtos.VisibilityLabelsService.newReflectiveService(this);
1077   }
1078 
1079   /****************************** VisibilityEndpoint service related methods ******************************/
1080   @Override
1081   public synchronized void addLabels(RpcController controller, VisibilityLabelsRequest request,
1082       RpcCallback<VisibilityLabelsResponse> done) {
1083     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
1084     List<VisibilityLabel> labels = request.getVisLabelList();
1085     if (!initialized) {
1086       setExceptionResults(labels.size(), new CoprocessorException(
1087           "VisibilityController not yet initialized"), response);
1088     }
1089     try {
1090       checkCallingUserAuth();
1091       List<Mutation> puts = new ArrayList<Mutation>(labels.size());
1092       RegionActionResult successResult = RegionActionResult.newBuilder().build();
1093       for (VisibilityLabel visLabel : labels) {
1094         byte[] label = visLabel.getLabel().toByteArray();
1095         String labelStr = Bytes.toString(label);
1096         if (VisibilityLabelsValidator.isValidLabel(label)) {
1097           if (this.visibilityManager.getLabelOrdinal(labelStr) > 0) {
1098             RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1099             failureResultBuilder.setException(ResponseConverter
1100                 .buildException(new LabelAlreadyExistsException("Label '" + labelStr
1101                     + "' already exists")));
1102             response.addResult(failureResultBuilder.build());
1103           } else {
1104             Put p = new Put(Bytes.toBytes(ordinalCounter));
1105             p.addImmutable(
1106                 LABELS_TABLE_FAMILY, LABEL_QUALIFIER, label, LABELS_TABLE_TAGS);
1107             if (LOG.isDebugEnabled()) {
1108               LOG.debug("Adding the label "+labelStr);
1109             }
1110             puts.add(p);
1111             ordinalCounter++;
1112             response.addResult(successResult);
1113           }
1114         } else {
1115           RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1116           failureResultBuilder.setException(ResponseConverter
1117               .buildException(new InvalidLabelException("Invalid visibility label '" + labelStr
1118                   + "'")));
1119           response.addResult(failureResultBuilder.build());
1120         }
1121       }
1122       OperationStatus[] opStatus = this.regionEnv.getRegion().batchMutate(
1123           puts.toArray(new Mutation[puts.size()]));
1124       int i = 0;
1125       for (OperationStatus status : opStatus) {
1126         if (status.getOperationStatusCode() != SUCCESS) {
1127           while (response.getResult(i) != successResult)
1128             i++;
1129           RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1130           failureResultBuilder.setException(ResponseConverter
1131               .buildException(new DoNotRetryIOException(status.getExceptionMsg())));
1132           response.setResult(i, failureResultBuilder.build());
1133         }
1134         i++;
1135       }
1136     } catch (IOException e) {
1137       LOG.error(e);
1138       setExceptionResults(labels.size(), e, response);
1139     }
1140     done.run(response.build());
1141   }
1142 
1143   private void setExceptionResults(int size, IOException e,
1144       VisibilityLabelsResponse.Builder response) {
1145     RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1146     failureResultBuilder.setException(ResponseConverter.buildException(e));
1147     RegionActionResult failureResult = failureResultBuilder.build();
1148     for (int i = 0; i < size; i++) {
1149       response.addResult(i, failureResult);
1150     }
1151   }
1152 
1153   private void performACLCheck() throws IOException {
1154     // Do ACL check only when the security is enabled.
1155     if (this.acOn && !isSystemOrSuperUser()) {
1156       User user = getActiveUser();
1157       throw new AccessDeniedException("User '" + (user != null ? user.getShortName() : "null")
1158           + " is not authorized to perform this action.");
1159     }
1160   }
1161 
1162   private List<List<Cell>> getExistingLabelsWithAuths() throws IOException {
1163     Scan scan = new Scan();
1164     RegionScanner scanner = this.regionEnv.getRegion().getScanner(scan);
1165     List<List<Cell>> existingLabels = new ArrayList<List<Cell>>();
1166     try {
1167       while (true) {
1168         List<Cell> cells = new ArrayList<Cell>();
1169         scanner.next(cells);
1170         if (cells.isEmpty()) {
1171           break;
1172         }
1173         existingLabels.add(cells);
1174       }
1175     } finally {
1176       scanner.close();
1177     }
1178     return existingLabels;
1179   }
1180 
1181   @Override
1182   public synchronized void setAuths(RpcController controller, SetAuthsRequest request,
1183       RpcCallback<VisibilityLabelsResponse> done) {
1184     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
1185     List<ByteString> auths = request.getAuthList();
1186     if (!initialized) {
1187       setExceptionResults(auths.size(), new CoprocessorException(
1188           "VisibilityController not yet initialized"), response);
1189     }
1190     byte[] user = request.getUser().toByteArray();
1191     try {
1192       checkCallingUserAuth();
1193       List<Mutation> puts = new ArrayList<Mutation>(auths.size());
1194       RegionActionResult successResult = RegionActionResult.newBuilder().build();
1195       for (ByteString authBS : auths) {
1196         byte[] auth = authBS.toByteArray();
1197         String authStr = Bytes.toString(auth);
1198         int labelOrdinal = this.visibilityManager.getLabelOrdinal(authStr);
1199         if (labelOrdinal == 0) {
1200           // This label is not yet added. 1st this should be added to the system
1201           RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1202           failureResultBuilder.setException(ResponseConverter
1203               .buildException(new InvalidLabelException("Label '" + authStr + "' doesn't exist")));
1204           response.addResult(failureResultBuilder.build());
1205         } else {
1206           Put p = new Put(Bytes.toBytes(labelOrdinal));
1207           p.addImmutable(
1208               LABELS_TABLE_FAMILY, user, DUMMY_VALUE, LABELS_TABLE_TAGS);
1209           puts.add(p);
1210           response.addResult(successResult);
1211         }
1212       }
1213       OperationStatus[] opStatus = this.regionEnv.getRegion().batchMutate(
1214           puts.toArray(new Mutation[puts.size()]));
1215       int i = 0;
1216       for (OperationStatus status : opStatus) {
1217         if (status.getOperationStatusCode() != SUCCESS) {
1218           while (response.getResult(i) != successResult) i++;
1219           RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1220           failureResultBuilder.setException(ResponseConverter
1221               .buildException(new DoNotRetryIOException(status.getExceptionMsg())));
1222           response.setResult(i, failureResultBuilder.build());
1223         }
1224         i++;
1225       }
1226     } catch (IOException e) {
1227       LOG.error(e);
1228       setExceptionResults(auths.size(), e, response);
1229     }
1230     done.run(response.build());
1231   }
1232 
1233   @Override
1234   public synchronized void getAuths(RpcController controller, GetAuthsRequest request,
1235       RpcCallback<GetAuthsResponse> done) {
1236     byte[] user = request.getUser().toByteArray();
1237     GetAuthsResponse.Builder response = GetAuthsResponse.newBuilder();
1238     response.setUser(request.getUser());
1239     try {
1240       List<String> labels = getUserAuthsFromLabelsTable(user);
1241       for (String label : labels) {
1242         response.addAuth(HBaseZeroCopyByteString.wrap(Bytes.toBytes(label)));
1243       }
1244     } catch (IOException e) {
1245       ResponseConverter.setControllerException(controller, e);
1246     }
1247     done.run(response.build());
1248   }
1249 
1250   private List<String> getUserAuthsFromLabelsTable(byte[] user) throws IOException {
1251     Scan s = new Scan();
1252     s.addColumn(LABELS_TABLE_FAMILY, user);
1253     Filter filter = createVisibilityLabelFilter(this.regionEnv.getRegion(), new Authorizations(
1254         SYSTEM_LABEL));
1255     s.setFilter(filter);
1256     List<String> auths = new ArrayList<String>();
1257     // We do ACL check here as we create scanner directly on region. It will not make calls to
1258     // AccessController CP methods.
1259     performACLCheck();
1260     RegionScanner scanner = this.regionEnv.getRegion().getScanner(s);
1261     List<Cell> results = new ArrayList<Cell>(1);
1262     while (true) {
1263       scanner.next(results);
1264       if (results.isEmpty()) break;
1265       Cell cell = results.get(0);
1266       int ordinal = Bytes.toInt(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
1267       String label = this.visibilityManager.getLabel(ordinal);
1268       if (label != null) {
1269         auths.add(label);
1270       }
1271       results.clear();
1272     }
1273     return auths;
1274   }
1275 
1276   @Override
1277   public synchronized void clearAuths(RpcController controller, SetAuthsRequest request,
1278       RpcCallback<VisibilityLabelsResponse> done) {
1279     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
1280     List<ByteString> auths = request.getAuthList();
1281     if (!initialized) {
1282       setExceptionResults(auths.size(), new CoprocessorException(
1283           "VisibilityController not yet initialized"), response);
1284     }
1285     byte[] user = request.getUser().toByteArray();
1286     try {
1287       checkCallingUserAuth();
1288       List<String> currentAuths = this.getUserAuthsFromLabelsTable(user);
1289       List<Mutation> deletes = new ArrayList<Mutation>(auths.size());
1290       RegionActionResult successResult = RegionActionResult.newBuilder().build();
1291       for (ByteString authBS : auths) {
1292         byte[] auth = authBS.toByteArray();
1293         String authStr = Bytes.toString(auth);
1294         if (currentAuths.contains(authStr)) {
1295           int labelOrdinal = this.visibilityManager.getLabelOrdinal(authStr);
1296           assert labelOrdinal > 0;
1297           Delete d = new Delete(Bytes.toBytes(labelOrdinal));
1298           d.deleteColumns(LABELS_TABLE_FAMILY, user);
1299           deletes.add(d);
1300           response.addResult(successResult);
1301         } else {
1302           // This label is not set for the user.
1303           RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1304           failureResultBuilder.setException(ResponseConverter
1305               .buildException(new InvalidLabelException("Label '" + authStr
1306                   + "' is not set for the user " + Bytes.toString(user))));
1307           response.addResult(failureResultBuilder.build());
1308         }
1309       }
1310       OperationStatus[] opStatus = this.regionEnv.getRegion().batchMutate(
1311           deletes.toArray(new Mutation[deletes.size()]));
1312       int i = 0;
1313       for (OperationStatus status : opStatus) {
1314         if (status.getOperationStatusCode() != SUCCESS) {
1315           while (response.getResult(i) != successResult) i++;
1316           RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
1317           failureResultBuilder.setException(ResponseConverter
1318               .buildException(new DoNotRetryIOException(status.getExceptionMsg())));
1319           response.setResult(i, failureResultBuilder.build());
1320         }
1321         i++;
1322       }
1323     } catch (IOException e) {
1324       LOG.error(e);
1325       setExceptionResults(auths.size(), e, response);
1326     }
1327     done.run(response.build());
1328   }
1329 
1330   private void checkCallingUserAuth() throws IOException {
1331     if (!this.acOn) {
1332       User user = getActiveUser();
1333       if (user == null) {
1334         throw new IOException("Unable to retrieve calling user");
1335       }
1336       List<String> auths = this.visibilityManager.getAuths(user.getShortName());
1337       if (LOG.isTraceEnabled()) {
1338         LOG.trace("The list of auths are "+auths);
1339       }
1340       if (!auths.contains(SYSTEM_LABEL)) {
1341         throw new AccessDeniedException("User '" + user.getShortName()
1342             + "' is not authorized to perform this action.");
1343       }
1344     }
1345   }
1346 }