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