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  
26  import java.io.IOException;
27  import java.net.InetAddress;
28  import java.util.ArrayList;
29  import java.util.HashMap;
30  import java.util.Iterator;
31  import java.util.List;
32  import java.util.Map;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.hbase.Cell;
38  import org.apache.hadoop.hbase.CellScanner;
39  import org.apache.hadoop.hbase.CellUtil;
40  import org.apache.hadoop.hbase.CoprocessorEnvironment;
41  import org.apache.hadoop.hbase.DoNotRetryIOException;
42  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
43  import org.apache.hadoop.hbase.HColumnDescriptor;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HTableDescriptor;
46  import org.apache.hadoop.hbase.KeyValue;
47  import org.apache.hadoop.hbase.KeyValue.Type;
48  import org.apache.hadoop.hbase.KeyValueUtil;
49  import org.apache.hadoop.hbase.TableName;
50  import org.apache.hadoop.hbase.Tag;
51  import org.apache.hadoop.hbase.TagType;
52  import org.apache.hadoop.hbase.catalog.MetaReader;
53  import org.apache.hadoop.hbase.classification.InterfaceAudience;
54  import org.apache.hadoop.hbase.client.Append;
55  import org.apache.hadoop.hbase.client.Delete;
56  import org.apache.hadoop.hbase.client.Get;
57  import org.apache.hadoop.hbase.client.Increment;
58  import org.apache.hadoop.hbase.client.Mutation;
59  import org.apache.hadoop.hbase.client.Put;
60  import org.apache.hadoop.hbase.client.Result;
61  import org.apache.hadoop.hbase.client.Scan;
62  import org.apache.hadoop.hbase.constraint.ConstraintException;
63  import org.apache.hadoop.hbase.coprocessor.BaseMasterAndRegionObserver;
64  import org.apache.hadoop.hbase.coprocessor.BaseRegionServerObserver;
65  import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
66  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
67  import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
68  import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
69  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
70  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
71  import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
72  import org.apache.hadoop.hbase.exceptions.DeserializationException;
73  import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
74  import org.apache.hadoop.hbase.filter.Filter;
75  import org.apache.hadoop.hbase.filter.FilterBase;
76  import org.apache.hadoop.hbase.filter.FilterList;
77  import org.apache.hadoop.hbase.io.hfile.HFile;
78  import org.apache.hadoop.hbase.ipc.RequestContext;
79  import org.apache.hadoop.hbase.master.MasterServices;
80  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
81  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
82  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos;
83  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
84  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
85  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
86  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
87  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
88  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
89  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
90  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
91  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
92  import org.apache.hadoop.hbase.regionserver.BloomType;
93  import org.apache.hadoop.hbase.regionserver.DeleteTracker;
94  import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
95  import org.apache.hadoop.hbase.regionserver.HRegion;
96  import org.apache.hadoop.hbase.regionserver.InternalScanner;
97  import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
98  import org.apache.hadoop.hbase.regionserver.OperationStatus;
99  import org.apache.hadoop.hbase.regionserver.RegionScanner;
100 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
101 import org.apache.hadoop.hbase.security.AccessDeniedException;
102 import org.apache.hadoop.hbase.security.User;
103 import org.apache.hadoop.hbase.security.access.AccessControlLists;
104 import org.apache.hadoop.hbase.security.access.AccessController;
105 import org.apache.hadoop.hbase.util.ByteStringer;
106 import org.apache.hadoop.hbase.util.Bytes;
107 import org.apache.hadoop.hbase.util.Pair;
108 
109 import com.google.common.collect.Lists;
110 import com.google.common.collect.MapMaker;
111 import com.google.protobuf.ByteString;
112 import com.google.protobuf.RpcCallback;
113 import com.google.protobuf.RpcController;
114 import com.google.protobuf.Service;
115 
116 /**
117  * Coprocessor that has both the MasterObserver and RegionObserver implemented that supports in
118  * visibility labels
119  */
120 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
121 public class VisibilityController extends BaseMasterAndRegionObserver implements
122     VisibilityLabelsService.Interface, CoprocessorService {
123 
124   private static final Log LOG = LogFactory.getLog(VisibilityController.class);
125   private static final Log AUDITLOG = LogFactory.getLog("SecurityLogger."
126       + VisibilityController.class.getName());
127   // flags if we are running on a region of the 'labels' table
128   private boolean labelsRegion = false;
129   // Flag denoting whether AcessController is available or not.
130   private boolean acOn = false;
131   private Configuration conf;
132   private volatile boolean initialized = false;
133   private boolean checkAuths = false;
134   /** Mapping of scanner instances to the user who created them */
135   private Map<InternalScanner,String> scannerOwners =
136       new MapMaker().weakKeys().makeMap();
137 
138   private List<String> superUsers;
139   private List<String> superGroups;
140   private VisibilityLabelService visibilityLabelService;
141 
142   // Add to this list if there are any reserved tag types
143   private static ArrayList<Byte> RESERVED_VIS_TAG_TYPES = new ArrayList<Byte>();
144   static {
145     RESERVED_VIS_TAG_TYPES.add(TagType.VISIBILITY_TAG_TYPE);
146     RESERVED_VIS_TAG_TYPES.add(TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE);
147     RESERVED_VIS_TAG_TYPES.add(TagType.STRING_VIS_TAG_TYPE);
148   }
149 
150   @Override
151   public void start(CoprocessorEnvironment env) throws IOException {
152     this.conf = env.getConfiguration();
153     if (HFile.getFormatVersion(conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
154       throw new RuntimeException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
155         + " is required to persist visibility labels. Consider setting " + HFile.FORMAT_VERSION_KEY
156         + " accordingly.");
157     }
158 
159     if (env instanceof RegionServerCoprocessorEnvironment) {
160       throw new RuntimeException("Visibility controller should not be configured as "
161           + "'hbase.coprocessor.regionserver.classes'.");
162     }
163     // Do not create for master CPs
164     if (!(env instanceof MasterCoprocessorEnvironment)) {
165       visibilityLabelService = VisibilityLabelServiceManager.getInstance()
166           .getVisibilityLabelService(this.conf);
167     }
168     Pair<List<String>, List<String>> superUsersAndGroups =
169         VisibilityUtils.getSystemAndSuperUsers(this.conf);
170     this.superUsers = superUsersAndGroups.getFirst();
171     this.superGroups = superUsersAndGroups.getSecond();
172   }
173 
174   @Override
175   public void stop(CoprocessorEnvironment env) throws IOException {
176 
177   }
178 
179   /********************************* Master related hooks **********************************/
180 
181   @Override
182   public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
183     // Need to create the new system table for labels here
184     MasterServices master = ctx.getEnvironment().getMasterServices();
185     if (!MetaReader.tableExists(master.getCatalogTracker(), LABELS_TABLE_NAME)) {
186       HTableDescriptor labelsTable = new HTableDescriptor(LABELS_TABLE_NAME);
187       HColumnDescriptor labelsColumn = new HColumnDescriptor(LABELS_TABLE_FAMILY);
188       labelsColumn.setBloomFilterType(BloomType.NONE);
189       labelsColumn.setBlockCacheEnabled(false); // We will cache all the labels. No need of normal
190                                                  // table block cache.
191       labelsTable.addFamily(labelsColumn);
192       // Let the "labels" table having only one region always. We are not expecting too many labels in
193       // the system.
194       labelsTable.setValue(HTableDescriptor.SPLIT_POLICY,
195           DisabledRegionSplitPolicy.class.getName());
196       labelsTable.setValue(Bytes.toBytes(HConstants.DISALLOW_WRITES_IN_RECOVERING),
197           Bytes.toBytes(true));
198       master.createTable(labelsTable, null);
199     }
200   }
201 
202   @Override
203   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
204       TableName tableName, HTableDescriptor htd) throws IOException {
205     if (LABELS_TABLE_NAME.equals(tableName)) {
206       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
207     }
208   }
209 
210   @Override
211   public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
212       HColumnDescriptor column) throws IOException {
213     if (LABELS_TABLE_NAME.equals(tableName)) {
214       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
215     }
216   }
217 
218   @Override
219   public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
220       TableName tableName, HColumnDescriptor descriptor) throws IOException {
221     if (LABELS_TABLE_NAME.equals(tableName)) {
222       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
223     }
224   }
225 
226   @Override
227   public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
228       TableName tableName, byte[] c) throws IOException {
229     if (LABELS_TABLE_NAME.equals(tableName)) {
230       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
231     }
232   }
233 
234   @Override
235   public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
236       throws IOException {
237     if (LABELS_TABLE_NAME.equals(tableName)) {
238       throw new ConstraintException("Cannot disable " + LABELS_TABLE_NAME);
239     }
240   }
241 
242   /****************************** Region related hooks ******************************/
243 
244   @Override
245   public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
246     // Read the entire labels table and populate the zk
247     if (e.getEnvironment().getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
248       this.labelsRegion = true;
249       this.acOn = CoprocessorHost.getLoadedCoprocessors().contains(AccessController.class.getName());
250       // Defer the init of VisibilityLabelService on labels region until it is in recovering state.
251       if (!e.getEnvironment().getRegion().isRecovering()) {
252         initVisibilityLabelService(e.getEnvironment());
253       }
254     } else {
255       checkAuths = e.getEnvironment().getConfiguration()
256           .getBoolean(VisibilityConstants.CHECK_AUTHS_FOR_MUTATION, false);
257       initVisibilityLabelService(e.getEnvironment());
258     }
259   }
260 
261   @Override
262   public void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e) {
263     if (this.labelsRegion) {
264       initVisibilityLabelService(e.getEnvironment());
265       LOG.debug("post labels region log replay");
266     }
267   }
268 
269   private void initVisibilityLabelService(RegionCoprocessorEnvironment env) {
270     try {
271       this.visibilityLabelService.init(env);
272       this.initialized = true;
273     } catch (IOException ioe) {
274       LOG.error("Error while initializing VisibilityLabelService..", ioe);
275       throw new RuntimeException(ioe);
276     }
277   }
278 
279   @Override
280   public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
281       MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
282     if (c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) {
283       return;
284     }
285     // TODO this can be made as a global LRU cache at HRS level?
286     Map<String, List<Tag>> labelCache = new HashMap<String, List<Tag>>();
287     for (int i = 0; i < miniBatchOp.size(); i++) {
288       Mutation m = miniBatchOp.getOperation(i);
289       CellVisibility cellVisibility = null;
290       try {
291         cellVisibility = m.getCellVisibility();
292       } catch (DeserializationException de) {
293         miniBatchOp.setOperationStatus(i,
294             new OperationStatus(SANITY_CHECK_FAILURE, de.getMessage()));
295         continue;
296       }
297       boolean sanityFailure = false;
298       boolean modifiedTagFound = false;
299       Pair<Boolean, Tag> pair = new Pair<Boolean, Tag>(false, null);
300       for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
301         pair = checkForReservedVisibilityTagPresence(cellScanner.current(), pair);
302         if (!pair.getFirst()) {
303           miniBatchOp.setOperationStatus(i, new OperationStatus(SANITY_CHECK_FAILURE,
304               "Mutation contains cell with reserved type tag"));
305           sanityFailure = true;
306           break;
307         } else {
308           // Indicates that the cell has a the tag which was modified in the src replication cluster
309           Tag tag = pair.getSecond();
310           if (cellVisibility == null && tag != null) {
311             // May need to store only the first one
312             cellVisibility = new CellVisibility(Bytes.toString(tag.getBuffer(), tag.getTagOffset(),
313                 tag.getTagLength()));
314             modifiedTagFound = true;
315           }
316         }
317       }
318       if (!sanityFailure) {
319         if (cellVisibility != null) {
320           String labelsExp = cellVisibility.getExpression();
321           List<Tag> visibilityTags = labelCache.get(labelsExp);
322           if (visibilityTags == null) {
323             // Don't check user auths for labels with Mutations when the user is super user
324             boolean authCheck = this.checkAuths && !(isSystemOrSuperUser());
325             try {
326               visibilityTags = this.visibilityLabelService.createVisibilityExpTags(labelsExp, true,
327                   authCheck);
328             } catch (InvalidLabelException e) {
329               miniBatchOp.setOperationStatus(i,
330                   new OperationStatus(SANITY_CHECK_FAILURE, e.getMessage()));
331             }
332             if (visibilityTags != null) {
333               labelCache.put(labelsExp, visibilityTags);
334             }
335           }
336           if (visibilityTags != null) {
337             List<Cell> updatedCells = new ArrayList<Cell>();
338             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
339               Cell cell = cellScanner.current();
340               List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
341                   cell.getTagsLengthUnsigned());
342               if (modifiedTagFound) {
343                 // Rewrite the tags by removing the modified tags.
344                 removeReplicationVisibilityTag(tags);
345               }
346               tags.addAll(visibilityTags);
347               Cell updatedCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
348                   cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(),
349                   cell.getFamilyLength(), cell.getQualifierArray(), cell.getQualifierOffset(),
350                   cell.getQualifierLength(), cell.getTimestamp(), Type.codeToType(cell
351                       .getTypeByte()), cell.getValueArray(), cell.getValueOffset(),
352                   cell.getValueLength(), tags);
353               updatedCells.add(updatedCell);
354             }
355             m.getFamilyCellMap().clear();
356             // Clear and add new Cells to the Mutation.
357             for (Cell cell : updatedCells) {
358               if (m instanceof Put) {
359                 Put p = (Put) m;
360                 p.add(cell);
361               } else if (m instanceof Delete) {
362                 // TODO : Cells without visibility tags would be handled in follow up issue
363                 Delete d = (Delete) m;
364                 d.addDeleteMarker(cell);
365               }
366             }
367           }
368         }
369       }
370     }
371   }
372 
373   @Override
374   public void prePrepareTimeStampForDeleteVersion(
375       ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation delete, Cell cell,
376       byte[] byteNow, Get get) throws IOException {
377     KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
378     CellVisibility cellVisibility = null;
379     try {
380       cellVisibility = delete.getCellVisibility();
381     } catch (DeserializationException de) {
382       throw new IOException("Invalid cell visibility specified " + delete, de);
383     }
384     // The check for checkForReservedVisibilityTagPresence happens in preBatchMutate happens.
385     // It happens for every mutation and that would be enough.
386     List<Tag> visibilityTags = new ArrayList<Tag>();
387     if (cellVisibility != null) {
388       String labelsExp = cellVisibility.getExpression();
389       try {
390         visibilityTags = this.visibilityLabelService.createVisibilityExpTags(labelsExp, false,
391             false);
392       } catch (InvalidLabelException e) {
393         throw new IOException("Invalid cell visibility specified " + labelsExp, e);
394       }
395     }
396     get.setFilter(new DeleteVersionVisibilityExpressionFilter(visibilityTags,
397         VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT));
398     List<Cell> result = ctx.getEnvironment().getRegion().get(get, false);
399 
400     if (result.size() < get.getMaxVersions()) {
401       // Nothing to delete
402       kv.updateLatestStamp(Bytes.toBytes(Long.MIN_VALUE));
403       return;
404     }
405     if (result.size() > get.getMaxVersions()) {
406       throw new RuntimeException("Unexpected size: " + result.size()
407           + ". Results more than the max versions obtained.");
408     }
409     KeyValue getkv = KeyValueUtil.ensureKeyValue(result.get(get.getMaxVersions() - 1));
410     Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(), getkv.getBuffer(),
411         getkv.getTimestampOffset(), Bytes.SIZEOF_LONG);
412     // We are bypassing here because in the HRegion.updateDeleteLatestVersionTimeStamp we would
413     // update with the current timestamp after again doing a get. As the hook as already determined
414     // the needed timestamp we need to bypass here.
415     // TODO : See if HRegion.updateDeleteLatestVersionTimeStamp() could be
416     // called only if the hook is not called.
417     ctx.bypass();
418   }
419 
420   /**
421    * Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE. This
422    * tag type is reserved and should not be explicitly set by user.
423    *
424    * @param cell
425    *          - the cell under consideration
426    * @param pair - an optional pair of type <Boolean, Tag> which would be reused
427    *               if already set and new one will be created if null is passed
428    * @return a pair<Boolean, Tag> - if the boolean is false then it indicates
429    *         that the cell has a RESERVERD_VIS_TAG and with boolean as true, not
430    *         null tag indicates that a string modified tag was found.
431    */
432   private Pair<Boolean, Tag> checkForReservedVisibilityTagPresence(Cell cell,
433       Pair<Boolean, Tag> pair) throws IOException {
434     if (pair == null) {
435       pair = new Pair<Boolean, Tag>(false, null);
436     } else {
437       pair.setFirst(false);
438       pair.setSecond(null);
439     }
440     // Bypass this check when the operation is done by a system/super user.
441     // This is done because, while Replication, the Cells coming to the peer cluster with reserved
442     // typed tags and this is fine and should get added to the peer cluster table
443     if (isSystemOrSuperUser()) {
444       // Does the cell contain special tag which indicates that the replicated
445       // cell visiblilty tags
446       // have been modified
447       Tag modifiedTag = null;
448       if (cell.getTagsLength() > 0) {
449         Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(),
450             cell.getTagsOffset(), cell.getTagsLength());
451         while (tagsIterator.hasNext()) {
452           Tag tag = tagsIterator.next();
453           if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
454             modifiedTag = tag;
455             break;
456           }
457         }
458       }
459       pair.setFirst(true);
460       pair.setSecond(modifiedTag);
461       return pair;
462     }
463     if (cell.getTagsLength() > 0) {
464       Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
465           cell.getTagsLength());
466       while (tagsItr.hasNext()) {
467         if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
468           return pair;
469         }
470       }
471     }
472     pair.setFirst(true);
473     return pair;
474   }
475 
476   /**
477    * Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE. This
478    * tag type is reserved and should not be explicitly set by user. There are
479    * two versions of this method one that accepts pair and other without pair.
480    * In case of preAppend and preIncrement the additional operations are not
481    * needed like checking for STRING_VIS_TAG_TYPE and hence the API without pair
482    * could be used.
483    *
484    * @param cell
485    * @return
486    * @throws IOException
487    */
488   private boolean checkForReservedVisibilityTagPresence(Cell cell) throws IOException {
489     // Bypass this check when the operation is done by a system/super user.
490     // This is done because, while Replication, the Cells coming to the peer
491     // cluster with reserved
492     // typed tags and this is fine and should get added to the peer cluster
493     // table
494     if (isSystemOrSuperUser()) {
495       return true;
496     }
497     if (cell.getTagsLengthUnsigned() > 0) {
498       Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
499           cell.getTagsLengthUnsigned());
500       while (tagsItr.hasNext()) {
501         if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
502           return false;
503         }
504       }
505     }
506     return true;
507   }
508 
509   private void removeReplicationVisibilityTag(List<Tag> tags) throws IOException {
510     Iterator<Tag> iterator = tags.iterator();
511     while (iterator.hasNext()) {
512       Tag tag = iterator.next();
513       if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
514         iterator.remove();
515         break;
516       }
517     }
518   }
519 
520   @Override
521   public RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan,
522       RegionScanner s) throws IOException {
523     if (!initialized) {
524       throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
525     }
526     HRegion region = e.getEnvironment().getRegion();
527     Authorizations authorizations = null;
528     try {
529       authorizations = scan.getAuthorizations();
530     } catch (DeserializationException de) {
531       throw new IOException(de);
532     }
533     if (authorizations == null) {
534       // No Authorizations present for this scan/Get!
535       // In case of system tables other than "labels" just scan with out visibility check and
536       // filtering. Checking visibility labels for META and NAMESPACE table is not needed.
537       TableName table = region.getRegionInfo().getTable();
538       if (table.isSystemTable() && !table.equals(LABELS_TABLE_NAME)) {
539         return s;
540       }
541     }
542 
543     Filter visibilityLabelFilter = VisibilityUtils.createVisibilityLabelFilter(region,
544         authorizations);
545     if (visibilityLabelFilter != null) {
546       Filter filter = scan.getFilter();
547       if (filter != null) {
548         scan.setFilter(new FilterList(filter, visibilityLabelFilter));
549       } else {
550         scan.setFilter(visibilityLabelFilter);
551       }
552     }
553     return s;
554   }
555 
556   @Override
557   public DeleteTracker postInstantiateDeleteTracker(
558       ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
559       throws IOException {
560     HRegion region = ctx.getEnvironment().getRegion();
561     TableName table = region.getRegionInfo().getTable();
562     if (table.isSystemTable()) {
563       return delTracker;
564     }
565     // We are creating a new type of delete tracker here which is able to track
566     // the timestamps and also the
567     // visibility tags per cell. The covering cells are determined not only
568     // based on the delete type and ts
569     // but also on the visibility expression matching.
570     return new VisibilityScanDeleteTracker();
571   }
572 
573   @Override
574   public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
575       final Scan scan, final RegionScanner s) throws IOException {
576     User user = VisibilityUtils.getActiveUser();
577     if (user != null && user.getShortName() != null) {
578       scannerOwners.put(s, user.getShortName());
579     }
580     return s;
581   }
582 
583   @Override
584   public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
585       final InternalScanner s, final List<Result> result, final int limit, final boolean hasNext)
586       throws IOException {
587     requireScannerOwner(s);
588     return hasNext;
589   }
590 
591   @Override
592   public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
593       final InternalScanner s) throws IOException {
594     requireScannerOwner(s);
595   }
596 
597   @Override
598   public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
599       final InternalScanner s) throws IOException {
600     // clean up any associated owner mapping
601     scannerOwners.remove(s);
602   }
603 
604   /**
605    * Verify, when servicing an RPC, that the caller is the scanner owner. If so, we assume that
606    * access control is correctly enforced based on the checks performed in preScannerOpen()
607    */
608   private void requireScannerOwner(InternalScanner s) throws AccessDeniedException {
609     if (RequestContext.isInRequestContext()) {
610       String requestUName = RequestContext.getRequestUserName();
611       String owner = scannerOwners.get(s);
612       if (owner != null && !owner.equals(requestUName)) {
613         throw new AccessDeniedException("User '" + requestUName + "' is not the scanner owner!");
614       }
615     }
616   }
617 
618   @Override
619   public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
620       throws IOException {
621     if (!initialized) {
622       throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
623     }
624     HRegion region = e.getEnvironment().getRegion();
625     Authorizations authorizations = null;
626     try {
627       authorizations = get.getAuthorizations();
628     } catch (DeserializationException de) {
629       throw new IOException(de);
630     }
631     if (authorizations == null) {
632       // No Authorizations present for this scan/Get!
633       // In case of system tables other than "labels" just scan with out visibility check and
634       // filtering. Checking visibility labels for META and NAMESPACE table is not needed.
635       TableName table = region.getRegionInfo().getTable();
636       if (table.isSystemTable() && !table.equals(LABELS_TABLE_NAME)) {
637         return;
638       }
639     }
640     Filter visibilityLabelFilter = VisibilityUtils.createVisibilityLabelFilter(e.getEnvironment()
641         .getRegion(), authorizations);
642     if (visibilityLabelFilter != null) {
643       Filter filter = get.getFilter();
644       if (filter != null) {
645         get.setFilter(new FilterList(filter, visibilityLabelFilter));
646       } else {
647         get.setFilter(visibilityLabelFilter);
648       }
649     }
650   }
651 
652   private boolean isSystemOrSuperUser() throws IOException {
653     User activeUser = VisibilityUtils.getActiveUser();
654     if (this.superUsers.contains(activeUser.getShortName())) {
655       return true;
656     }
657     String[] groups = activeUser.getGroupNames();
658     if (groups != null && groups.length > 0) {
659       for (String group : groups) {
660         if (this.superGroups.contains(group)) {
661           return true;
662         }
663       }
664     }
665     return false;
666   }
667 
668   @Override
669   public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> e, Append append)
670       throws IOException {
671     for (CellScanner cellScanner = append.cellScanner(); cellScanner.advance();) {
672       if (!checkForReservedVisibilityTagPresence(cellScanner.current())) {
673         throw new FailedSanityCheckException("Append contains cell with reserved type tag");
674       }
675     }
676     return null;
677   }
678 
679   @Override
680   public Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> e, Increment increment)
681       throws IOException {
682     for (CellScanner cellScanner = increment.cellScanner(); cellScanner.advance();) {
683       if (!checkForReservedVisibilityTagPresence(cellScanner.current())) {
684         throw new FailedSanityCheckException("Increment contains cell with reserved type tag");
685       }
686     }
687     return null;
688   }
689 
690   @Override
691   public Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
692       MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
693     List<Tag> tags = Lists.newArrayList();
694     CellVisibility cellVisibility = null;
695     try {
696       cellVisibility = mutation.getCellVisibility();
697     } catch (DeserializationException e) {
698       throw new IOException(e);
699     }
700     if (cellVisibility == null) {
701       return newCell;
702     }
703     // Prepend new visibility tags to a new list of tags for the cell
704     // Don't check user auths for labels with Mutations when the user is super user
705     boolean authCheck = this.checkAuths && !(isSystemOrSuperUser());
706     tags.addAll(this.visibilityLabelService.createVisibilityExpTags(cellVisibility.getExpression(),
707         true, authCheck));
708     // Save an object allocation where we can
709     if (newCell.getTagsLengthUnsigned() > 0) {
710       // Carry forward all other tags
711       Iterator<Tag> tagsItr = CellUtil.tagsIterator(newCell.getTagsArray(),
712           newCell.getTagsOffset(), newCell.getTagsLengthUnsigned());
713       while (tagsItr.hasNext()) {
714         Tag tag = tagsItr.next();
715         if (tag.getType() != TagType.VISIBILITY_TAG_TYPE
716             && tag.getType() != TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
717           tags.add(tag);
718         }
719       }
720     }
721 
722     // We need to create another KV, unfortunately, because the current new KV
723     // has no space for tags
724     KeyValue rewriteKv = new KeyValue(newCell.getRowArray(), newCell.getRowOffset(),
725         newCell.getRowLength(), newCell.getFamilyArray(), newCell.getFamilyOffset(),
726         newCell.getFamilyLength(), newCell.getQualifierArray(), newCell.getQualifierOffset(),
727         newCell.getQualifierLength(), newCell.getTimestamp(), KeyValue.Type.codeToType(newCell
728             .getTypeByte()), newCell.getValueArray(), newCell.getValueOffset(),
729         newCell.getValueLength(), tags);
730     // Preserve mvcc data
731     rewriteKv.setMvccVersion(newCell.getMvccVersion());
732     return rewriteKv;
733   }
734 
735   @Override
736   public Service getService() {
737     return VisibilityLabelsProtos.VisibilityLabelsService.newReflectiveService(this);
738   }
739 
740   /****************************** VisibilityEndpoint service related methods ******************************/
741   @Override
742   public synchronized void addLabels(RpcController controller, VisibilityLabelsRequest request,
743       RpcCallback<VisibilityLabelsResponse> done) {
744     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
745     List<VisibilityLabel> visLabels = request.getVisLabelList();
746     if (!initialized) {
747       setExceptionResults(visLabels.size(),
748         new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"),
749         response);
750     } else {
751       List<byte[]> labels = new ArrayList<byte[]>(visLabels.size());
752       try {
753         checkCallingUserAuth();
754         RegionActionResult successResult = RegionActionResult.newBuilder().build();
755         for (VisibilityLabel visLabel : visLabels) {
756           byte[] label = visLabel.getLabel().toByteArray();
757           labels.add(label);
758           response.addResult(successResult); // Just mark as success. Later it will get reset
759                                              // based on the result from
760                                              // visibilityLabelService.addLabels ()
761         }
762         if (!labels.isEmpty()) {
763           OperationStatus[] opStatus = this.visibilityLabelService.addLabels(labels);
764           logResult(true, "addLabels", "Adding labels allowed", null, labels, null);
765           int i = 0;
766           for (OperationStatus status : opStatus) {
767             while (response.getResult(i) != successResult)
768               i++;
769             if (status.getOperationStatusCode() != SUCCESS) {
770               RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
771               failureResultBuilder.setException(ResponseConverter
772                   .buildException(new DoNotRetryIOException(status.getExceptionMsg())));
773               response.setResult(i, failureResultBuilder.build());
774             }
775             i++;
776           }
777         }
778       } catch (AccessDeniedException e) {
779         logResult(false, "addLabels", e.getMessage(), null, labels, null);
780         LOG.error("User is not having required permissions to add labels", e);
781         setExceptionResults(visLabels.size(), e, response);
782       } catch (IOException e) {
783         LOG.error(e);
784         setExceptionResults(visLabels.size(), e, response);
785       }
786     }
787     done.run(response.build());
788   }
789 
790   private void setExceptionResults(int size, IOException e,
791       VisibilityLabelsResponse.Builder response) {
792     RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
793     failureResultBuilder.setException(ResponseConverter.buildException(e));
794     RegionActionResult failureResult = failureResultBuilder.build();
795     for (int i = 0; i < size; i++) {
796       response.addResult(i, failureResult);
797     }
798   }
799 
800   @Override
801   public synchronized void setAuths(RpcController controller, SetAuthsRequest request,
802       RpcCallback<VisibilityLabelsResponse> done) {
803     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
804     List<ByteString> auths = request.getAuthList();
805     if (!initialized) {
806       setExceptionResults(auths.size(),
807         new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"),
808         response);
809     } else {
810       byte[] user = request.getUser().toByteArray();
811       List<byte[]> labelAuths = new ArrayList<byte[]>(auths.size());
812       try {
813         checkCallingUserAuth();
814 
815         for (ByteString authBS : auths) {
816           labelAuths.add(authBS.toByteArray());
817         }
818         OperationStatus[] opStatus = this.visibilityLabelService.setAuths(user, labelAuths);
819         logResult(true, "setAuths", "Setting authorization for labels allowed", user, labelAuths,
820           null);
821         RegionActionResult successResult = RegionActionResult.newBuilder().build();
822         for (OperationStatus status : opStatus) {
823           if (status.getOperationStatusCode() == SUCCESS) {
824             response.addResult(successResult);
825           } else {
826             RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
827             failureResultBuilder.setException(ResponseConverter
828                 .buildException(new DoNotRetryIOException(status.getExceptionMsg())));
829             response.addResult(failureResultBuilder.build());
830           }
831         }
832       } catch (AccessDeniedException e) {
833         logResult(false, "setAuths", e.getMessage(), user, labelAuths, null);
834         LOG.error("User is not having required permissions to set authorization", e);
835         setExceptionResults(auths.size(), e, response);
836       } catch (IOException e) {
837         LOG.error(e);
838         setExceptionResults(auths.size(), e, response);
839       }
840     }
841     done.run(response.build());
842   }
843 
844   private void logResult(boolean isAllowed, String request, String reason, byte[] user,
845       List<byte[]> labelAuths, String regex) {
846     if (AUDITLOG.isTraceEnabled()) {
847       RequestContext ctx = RequestContext.get();
848       InetAddress remoteAddr = null;
849       if (ctx != null) {
850         remoteAddr = ctx.getRemoteAddress();
851       }
852 
853       List<String> labelAuthsStr = new ArrayList<String>();
854       if (labelAuths != null) {
855         int labelAuthsSize = labelAuths.size();
856         labelAuthsStr = new ArrayList<String>(labelAuthsSize);
857         for (int i = 0; i < labelAuthsSize; i++) {
858           labelAuthsStr.add(Bytes.toString(labelAuths.get(i)));
859         }
860       }
861 
862       User requestingUser = null;
863       try {
864         requestingUser = VisibilityUtils.getActiveUser();
865       } catch (IOException e) {
866         LOG.warn("Failed to get active system user.");
867         LOG.debug("Details on failure to get active system user.", e);
868       }
869       AUDITLOG.trace("Access " + (isAllowed ? "allowed" : "denied") + " for user "
870           + (requestingUser != null ? requestingUser.getShortName() : "UNKNOWN") + "; reason: "
871           + reason + "; remote address: " + (remoteAddr != null ? remoteAddr : "") + "; request: "
872           + request + "; user: " + (user != null ? Bytes.toShort(user) : "null") + "; labels: "
873           + labelAuthsStr + "; regex: " + regex);
874     }
875   }
876 
877   @Override
878   public synchronized void getAuths(RpcController controller, GetAuthsRequest request,
879       RpcCallback<GetAuthsResponse> done) {
880     GetAuthsResponse.Builder response = GetAuthsResponse.newBuilder();
881     if (!initialized) {
882       controller.setFailed("VisibilityController not yet initialized");
883     } else {
884       byte[] user = request.getUser().toByteArray();
885       List<String> labels = null;
886       try {
887         // We do ACL check here as we create scanner directly on region. It will not make calls to
888         // AccessController CP methods.
889         if (this.acOn && !isSystemOrSuperUser()) {
890           User requestingUser = VisibilityUtils.getActiveUser();
891           throw new AccessDeniedException("User '"
892               + (requestingUser != null ? requestingUser.getShortName() : "null")
893               + "' is not authorized to perform this action.");
894         }
895         if (AccessControlLists.isGroupPrincipal(Bytes.toString(user))) {
896           // For backward compatibility. Previous custom visibilityLabelService
897           // implementation may not have getGroupAuths
898           try {
899             this.visibilityLabelService.getClass().getDeclaredMethod("getGroupAuths",
900               new Class[] { String[].class, Boolean.TYPE });
901           } catch (SecurityException e) {
902             throw new AccessDeniedException("Failed to obtain getGroupAuths implementation");
903           } catch (NoSuchMethodException e) {
904             throw new AccessDeniedException(
905                 "Get group auth is not supported in this implementation");
906           }
907           String group = AccessControlLists.getGroupName(Bytes.toString(user));
908           labels = this.visibilityLabelService.getGroupAuths(new String[]{group}, false);
909         }
910         else {
911           labels = this.visibilityLabelService.getAuths(user, false);
912         }
913         logResult(true, "getAuths", "Get authorizations for user allowed", user, null, null);
914       } catch (AccessDeniedException e) {
915         logResult(false, "getAuths", e.getMessage(), user, null, null);
916         ResponseConverter.setControllerException(controller, e);
917       } catch (IOException e) {
918         ResponseConverter.setControllerException(controller, e);
919       }
920       response.setUser(request.getUser());
921       if (labels != null) {
922         for (String label : labels) {
923           response.addAuth(ByteStringer.wrap(Bytes.toBytes(label)));
924         }
925       }
926     }
927     done.run(response.build());
928   }
929 
930   @Override
931   public synchronized void clearAuths(RpcController controller, SetAuthsRequest request,
932       RpcCallback<VisibilityLabelsResponse> done) {
933     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
934     List<ByteString> auths = request.getAuthList();
935     if (!initialized) {
936       setExceptionResults(auths.size(), new CoprocessorException(
937           "VisibilityController not yet initialized"), response);
938     } else {
939       byte[] requestUser = request.getUser().toByteArray();
940       List<byte[]> labelAuths = new ArrayList<byte[]>(auths.size());
941       try {
942         // When AC is ON, do AC based user auth check
943         if (this.acOn && !isSystemOrSuperUser()) {
944           User user = VisibilityUtils.getActiveUser();
945           throw new AccessDeniedException("User '" + (user != null ? user.getShortName() : "null")
946               + " is not authorized to perform this action.");
947         }
948         checkCallingUserAuth(); // When AC is not in place the calling user should have SYSTEM_LABEL
949                                 // auth to do this action.
950         for (ByteString authBS : auths) {
951           labelAuths.add(authBS.toByteArray());
952         }
953 
954         OperationStatus[] opStatus =
955             this.visibilityLabelService.clearAuths(requestUser, labelAuths);
956         logResult(true, "clearAuths", "Removing authorization for labels allowed", requestUser,
957           labelAuths, null);
958         RegionActionResult successResult = RegionActionResult.newBuilder().build();
959         for (OperationStatus status : opStatus) {
960           if (status.getOperationStatusCode() == SUCCESS) {
961             response.addResult(successResult);
962           } else {
963             RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
964             failureResultBuilder.setException(ResponseConverter
965                 .buildException(new DoNotRetryIOException(status.getExceptionMsg())));
966             response.addResult(failureResultBuilder.build());
967           }
968         }
969       } catch (AccessDeniedException e) {
970         logResult(false, "clearAuths", e.getMessage(), requestUser, labelAuths, null);
971         LOG.error("User is not having required permissions to clear authorization", e);
972         setExceptionResults(auths.size(), e, response);
973       } catch (IOException e) {
974         LOG.error(e);
975         setExceptionResults(auths.size(), e, response);
976       }
977     }
978     done.run(response.build());
979   }
980 
981   @Override
982   public synchronized void listLabels(RpcController controller, ListLabelsRequest request,
983       RpcCallback<ListLabelsResponse> done) {
984     ListLabelsResponse.Builder response = ListLabelsResponse.newBuilder();
985     if (!initialized) {
986       controller.setFailed("VisibilityController not yet initialized");
987     } else {
988       List<String> labels = null;
989       String regex = request.hasRegex() ? request.getRegex() : null;
990       try {
991         // We do ACL check here as we create scanner directly on region. It will not make calls to
992         // AccessController CP methods.
993         if (this.acOn && !isSystemOrSuperUser()) {
994           User requestingUser = VisibilityUtils.getActiveUser();
995           throw new AccessDeniedException("User '"
996               + (requestingUser != null ? requestingUser.getShortName() : "null")
997               + "' is not authorized to perform this action.");
998         }
999         labels = this.visibilityLabelService.listLabels(regex);
1000         logResult(false, "listLabels", "Listing labels allowed", null, null, regex);
1001       } catch (AccessDeniedException e) {
1002         logResult(false, "listLabels", e.getMessage(), null, null, regex);
1003         ResponseConverter.setControllerException(controller, e);
1004       } catch (IOException e) {
1005         ResponseConverter.setControllerException(controller, e);
1006       }
1007       if (labels != null && !labels.isEmpty()) {
1008         for (String label : labels) {
1009           response.addLabel(ByteStringer.wrap(Bytes.toBytes(label)));
1010         }
1011       }
1012     }
1013     done.run(response.build());
1014   }
1015 
1016   private void checkCallingUserAuth() throws IOException {
1017     if (!this.acOn) {
1018       User user = VisibilityUtils.getActiveUser();
1019       if (user == null) {
1020         throw new IOException("Unable to retrieve calling user");
1021       }
1022 
1023       boolean havingSystemAuth = false;
1024       try {
1025         this.visibilityLabelService.getClass().getDeclaredMethod("havingSystemAuth",
1026           new Class[] { User.class });
1027         havingSystemAuth = this.visibilityLabelService.havingSystemAuth(user);
1028       } catch (SecurityException e) {
1029         // Just consider this as AccessDeniedException
1030       } catch (NoSuchMethodException e) {
1031         // VLS not having havingSystemAuth(User) method. Go with deprecated havingSystemAuth(byte[])
1032         // method invoke
1033         havingSystemAuth = this.visibilityLabelService.havingSystemAuth(Bytes.toBytes(user
1034           .getShortName()));
1035       }
1036       if (!havingSystemAuth) {
1037         throw new AccessDeniedException("User '" + user.getShortName()
1038           + "' is not authorized to perform this action.");
1039       }
1040     }
1041   }
1042 
1043   private static class DeleteVersionVisibilityExpressionFilter extends FilterBase {
1044     private List<Tag> deleteCellVisTags;
1045     private Byte deleteCellVisTagsFormat;
1046 
1047     public DeleteVersionVisibilityExpressionFilter(List<Tag> deleteCellVisTags,
1048         Byte deleteCellVisTagsFormat) {
1049       this.deleteCellVisTags = deleteCellVisTags;
1050       this.deleteCellVisTagsFormat = deleteCellVisTagsFormat;
1051     }
1052 
1053     @Override
1054     public ReturnCode filterKeyValue(Cell cell) throws IOException {
1055       List<Tag> putVisTags = new ArrayList<Tag>();
1056       Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);
1057       boolean matchFound = VisibilityLabelServiceManager
1058           .getInstance().getVisibilityLabelService()
1059           .matchVisibility(putVisTags, putCellVisTagsFormat, deleteCellVisTags,
1060               deleteCellVisTagsFormat);
1061       return matchFound ? ReturnCode.INCLUDE : ReturnCode.SKIP;
1062     }
1063   }
1064 
1065   @Override
1066   public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
1067       TableName tableName) throws IOException {
1068   }
1069 
1070   @Override
1071   public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
1072       TableName tableName) throws IOException {
1073   }
1074 
1075   @Override
1076   public void preTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
1077       TableName tableName) throws IOException {
1078   }
1079 
1080   @Override
1081   public void postTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
1082       TableName tableName) throws IOException {
1083   }
1084 
1085   /**
1086    * A RegionServerObserver impl that provides the custom
1087    * VisibilityReplicationEndpoint. This class should be configured as the
1088    * 'hbase.coprocessor.regionserver.classes' for the visibility tags to be
1089    * replicated as string.  The value for the configuration should be
1090    * 'org.apache.hadoop.hbase.security.visibility.VisibilityController$VisibilityReplication'.
1091    */
1092   public static class VisibilityReplication extends BaseRegionServerObserver {
1093     private Configuration conf;
1094     private VisibilityLabelService visibilityLabelService;
1095 
1096     @Override
1097     public void start(CoprocessorEnvironment env) throws IOException {
1098       this.conf = env.getConfiguration();
1099       visibilityLabelService = VisibilityLabelServiceManager.getInstance()
1100           .getVisibilityLabelService(this.conf);
1101     }
1102 
1103     @Override
1104     public void stop(CoprocessorEnvironment env) throws IOException {
1105     }
1106 
1107     @Override
1108     public ReplicationEndpoint postCreateReplicationEndPoint(
1109         ObserverContext<RegionServerCoprocessorEnvironment> ctx, ReplicationEndpoint endpoint) {
1110       return new VisibilityReplicationEndpoint(endpoint, visibilityLabelService);
1111     }
1112   }
1113 }