001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.security.visibility;
019
020import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_FAMILY;
021import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
022import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABEL_QUALIFIER;
023import static org.junit.Assert.assertEquals;
024import static org.junit.Assert.assertFalse;
025import static org.junit.Assert.assertNotNull;
026import static org.junit.Assert.assertNull;
027import static org.junit.Assert.assertTrue;
028import static org.junit.Assert.fail;
029
030import java.io.IOException;
031import java.security.PrivilegedExceptionAction;
032import java.util.ArrayList;
033import java.util.Collection;
034import java.util.List;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.hbase.Cell;
037import org.apache.hadoop.hbase.CellScanner;
038import org.apache.hadoop.hbase.HBaseTestingUtil;
039import org.apache.hadoop.hbase.HConstants;
040import org.apache.hadoop.hbase.TableName;
041import org.apache.hadoop.hbase.client.Admin;
042import org.apache.hadoop.hbase.client.Append;
043import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
044import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
045import org.apache.hadoop.hbase.client.Connection;
046import org.apache.hadoop.hbase.client.ConnectionFactory;
047import org.apache.hadoop.hbase.client.Get;
048import org.apache.hadoop.hbase.client.Increment;
049import org.apache.hadoop.hbase.client.Put;
050import org.apache.hadoop.hbase.client.Result;
051import org.apache.hadoop.hbase.client.ResultScanner;
052import org.apache.hadoop.hbase.client.RowMutations;
053import org.apache.hadoop.hbase.client.Scan;
054import org.apache.hadoop.hbase.client.Table;
055import org.apache.hadoop.hbase.client.TableDescriptor;
056import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
057import org.apache.hadoop.hbase.client.security.SecurityCapability;
058import org.apache.hadoop.hbase.regionserver.BloomType;
059import org.apache.hadoop.hbase.regionserver.HRegion;
060import org.apache.hadoop.hbase.regionserver.HRegionServer;
061import org.apache.hadoop.hbase.regionserver.HStore;
062import org.apache.hadoop.hbase.regionserver.HStoreFile;
063import org.apache.hadoop.hbase.security.User;
064import org.apache.hadoop.hbase.util.Bytes;
065import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
066import org.junit.After;
067import org.junit.AfterClass;
068import org.junit.Rule;
069import org.junit.Test;
070import org.junit.rules.TestName;
071
072import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
073
074import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
075import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
076import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
077
078/**
079 * Base test class for visibility labels basic features
080 */
081public abstract class TestVisibilityLabels {
082
083  public static final String TOPSECRET = "topsecret";
084  public static final String PUBLIC = "public";
085  public static final String PRIVATE = "private";
086  public static final String CONFIDENTIAL = "confidential";
087  public static final String SECRET = "secret";
088  public static final String COPYRIGHT = "\u00A9ABC";
089  public static final String ACCENT = "\u0941";
090  public static final String UNICODE_VIS_TAG =
091    COPYRIGHT + "\"" + ACCENT + "\\" + SECRET + "\"" + "\u0027&\\";
092  public static final String UC1 = "\u0027\"\u002b";
093  public static final String UC2 = "\u002d\u003f";
094  public static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
095  public static final byte[] row1 = Bytes.toBytes("row1");
096  public static final byte[] row2 = Bytes.toBytes("row2");
097  public static final byte[] row3 = Bytes.toBytes("row3");
098  public static final byte[] row4 = Bytes.toBytes("row4");
099  public final static byte[] fam = Bytes.toBytes("info");
100  public final static byte[] qual = Bytes.toBytes("qual");
101  public final static byte[] value = Bytes.toBytes("value");
102  public static Configuration conf;
103
104  private volatile boolean killedRS = false;
105  @Rule
106  public final TestName TEST_NAME = new TestName();
107  public static User SUPERUSER, USER1;
108
109  @AfterClass
110  public static void tearDownAfterClass() throws Exception {
111    TEST_UTIL.shutdownMiniCluster();
112  }
113
114  @After
115  public void tearDown() throws Exception {
116    killedRS = false;
117  }
118
119  @Test
120  public void testSecurityCapabilities() throws Exception {
121    List<SecurityCapability> capabilities =
122      TEST_UTIL.getConnection().getAdmin().getSecurityCapabilities();
123    assertTrue("CELL_VISIBILITY capability is missing",
124      capabilities.contains(SecurityCapability.CELL_VISIBILITY));
125  }
126
127  @Test
128  public void testSimpleVisibilityLabels() throws Exception {
129    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
130    try (Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "|" + CONFIDENTIAL,
131      PRIVATE + "|" + CONFIDENTIAL)) {
132      Scan s = new Scan();
133      s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL, PRIVATE));
134      ResultScanner scanner = table.getScanner(s);
135      Result[] next = scanner.next(3);
136
137      assertTrue(next.length == 2);
138      CellScanner cellScanner = next[0].cellScanner();
139      cellScanner.advance();
140      Cell current = cellScanner.current();
141      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
142        row1, 0, row1.length));
143      cellScanner = next[1].cellScanner();
144      cellScanner.advance();
145      current = cellScanner.current();
146      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
147        row2, 0, row2.length));
148    }
149  }
150
151  @Test
152  public void testSimpleVisibilityLabelsWithUniCodeCharacters() throws Exception {
153    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
154    try (Table table = createTableAndWriteDataWithLabels(tableName,
155      SECRET + "|" + CellVisibility.quote(COPYRIGHT), "(" + CellVisibility.quote(COPYRIGHT) + "&"
156        + CellVisibility.quote(ACCENT) + ")|" + CONFIDENTIAL,
157      CellVisibility.quote(UNICODE_VIS_TAG) + "&" + SECRET)) {
158      Scan s = new Scan();
159      s.setAuthorizations(
160        new Authorizations(SECRET, CONFIDENTIAL, PRIVATE, COPYRIGHT, ACCENT, UNICODE_VIS_TAG));
161      ResultScanner scanner = table.getScanner(s);
162      Result[] next = scanner.next(3);
163      assertTrue(next.length == 3);
164      CellScanner cellScanner = next[0].cellScanner();
165      cellScanner.advance();
166      Cell current = cellScanner.current();
167      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
168        row1, 0, row1.length));
169      cellScanner = next[1].cellScanner();
170      cellScanner.advance();
171      current = cellScanner.current();
172      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
173        row2, 0, row2.length));
174      cellScanner = next[2].cellScanner();
175      cellScanner.advance();
176      current = cellScanner.current();
177      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
178        row3, 0, row3.length));
179    }
180  }
181
182  @Test
183  public void testAuthorizationsWithSpecialUnicodeCharacters() throws Exception {
184    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
185    try (Table table = createTableAndWriteDataWithLabels(tableName,
186      CellVisibility.quote(UC1) + "|" + CellVisibility.quote(UC2), CellVisibility.quote(UC1),
187      CellVisibility.quote(UNICODE_VIS_TAG))) {
188      Scan s = new Scan();
189      s.setAuthorizations(new Authorizations(UC1, UC2, ACCENT, UNICODE_VIS_TAG));
190      ResultScanner scanner = table.getScanner(s);
191      Result[] next = scanner.next(3);
192      assertTrue(next.length == 3);
193      CellScanner cellScanner = next[0].cellScanner();
194      cellScanner.advance();
195      Cell current = cellScanner.current();
196      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
197        row1, 0, row1.length));
198      cellScanner = next[1].cellScanner();
199      cellScanner.advance();
200      current = cellScanner.current();
201      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
202        row2, 0, row2.length));
203      cellScanner = next[2].cellScanner();
204      cellScanner.advance();
205      current = cellScanner.current();
206      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
207        row3, 0, row3.length));
208    }
209  }
210
211  @Test
212  public void testVisibilityLabelsWithComplexLabels() throws Exception {
213    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
214    try (Table table = createTableAndWriteDataWithLabels(tableName,
215      "(" + SECRET + "|" + CONFIDENTIAL + ")" + "&" + "!" + TOPSECRET,
216      "(" + PRIVATE + "&" + CONFIDENTIAL + "&" + SECRET + ")",
217      "(" + PRIVATE + "&" + CONFIDENTIAL + "&" + SECRET + ")",
218      "(" + PRIVATE + "&" + CONFIDENTIAL + "&" + SECRET + ")")) {
219      Scan s = new Scan();
220      s.setAuthorizations(new Authorizations(TOPSECRET, CONFIDENTIAL, PRIVATE, PUBLIC, SECRET));
221      ResultScanner scanner = table.getScanner(s);
222      Result[] next = scanner.next(4);
223      assertEquals(3, next.length);
224      CellScanner cellScanner = next[0].cellScanner();
225      cellScanner.advance();
226      Cell current = cellScanner.current();
227      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
228        row2, 0, row2.length));
229      cellScanner = next[1].cellScanner();
230      cellScanner.advance();
231      current = cellScanner.current();
232      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
233        row3, 0, row3.length));
234      cellScanner = next[2].cellScanner();
235      cellScanner.advance();
236      current = cellScanner.current();
237      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
238        row4, 0, row4.length));
239    }
240  }
241
242  @Test
243  public void testVisibilityLabelsThatDoesNotPassTheCriteria() throws Exception {
244    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
245    try (Table table = createTableAndWriteDataWithLabels(tableName,
246      "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE)) {
247      Scan s = new Scan();
248      s.setAuthorizations(new Authorizations(PUBLIC));
249      ResultScanner scanner = table.getScanner(s);
250      Result[] next = scanner.next(3);
251      assertTrue(next.length == 0);
252    }
253  }
254
255  @Test
256  public void testVisibilityLabelsInPutsThatDoesNotMatchAnyDefinedLabels() throws Exception {
257    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
258    try {
259      createTableAndWriteDataWithLabels(tableName, "SAMPLE_LABEL", "TEST");
260      fail("Should have failed with failed sanity check exception");
261    } catch (Exception e) {
262    }
263  }
264
265  @Test
266  public void testVisibilityLabelsInScanThatDoesNotMatchAnyDefinedLabels() throws Exception {
267    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
268    try (Table table = createTableAndWriteDataWithLabels(tableName,
269      "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE)) {
270      Scan s = new Scan();
271      s.setAuthorizations(new Authorizations("SAMPLE"));
272      ResultScanner scanner = table.getScanner(s);
273      Result[] next = scanner.next(3);
274      assertTrue(next.length == 0);
275    }
276  }
277
278  @Test
279  public void testVisibilityLabelsWithGet() throws Exception {
280    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
281    try (Table table = createTableAndWriteDataWithLabels(tableName,
282      SECRET + "&" + CONFIDENTIAL + "&!" + PRIVATE, SECRET + "&" + CONFIDENTIAL + "&" + PRIVATE)) {
283      Get get = new Get(row1);
284      get.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL));
285      Result result = table.get(get);
286      assertTrue(!result.isEmpty());
287      Cell cell = result.getColumnLatestCell(fam, qual);
288      assertTrue(Bytes.equals(value, 0, value.length, cell.getValueArray(), cell.getValueOffset(),
289        cell.getValueLength()));
290    }
291  }
292
293  @Test
294  public void testVisibilityLabelsOnKillingOfRSContainingLabelsTable() throws Exception {
295    List<RegionServerThread> regionServerThreads =
296      TEST_UTIL.getHBaseCluster().getRegionServerThreads();
297    int liveRS = 0;
298    for (RegionServerThread rsThreads : regionServerThreads) {
299      if (!rsThreads.getRegionServer().isAborted()) {
300        liveRS++;
301      }
302    }
303    if (liveRS == 1) {
304      TEST_UTIL.getHBaseCluster().startRegionServer();
305    }
306    Thread t1 = new Thread() {
307      @Override
308      public void run() {
309        List<RegionServerThread> regionServerThreads =
310          TEST_UTIL.getHBaseCluster().getRegionServerThreads();
311        for (RegionServerThread rsThread : regionServerThreads) {
312          List<HRegion> onlineRegions = rsThread.getRegionServer().getRegions(LABELS_TABLE_NAME);
313          if (onlineRegions.size() > 0) {
314            rsThread.getRegionServer().abort("Aborting ");
315            killedRS = true;
316            break;
317          }
318        }
319      }
320
321    };
322    t1.start();
323    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
324    Thread t = new Thread() {
325      @Override
326      public void run() {
327        try {
328          while (!killedRS) {
329            Thread.sleep(1);
330          }
331          createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + ")",
332            PRIVATE);
333        } catch (Exception e) {
334        }
335      }
336    };
337    t.start();
338    regionServerThreads = TEST_UTIL.getHBaseCluster().getRegionServerThreads();
339    while (!killedRS) {
340      Thread.sleep(10);
341    }
342    regionServerThreads = TEST_UTIL.getHBaseCluster().getRegionServerThreads();
343    for (RegionServerThread rsThread : regionServerThreads) {
344      while (true) {
345        if (!rsThread.getRegionServer().isAborted()) {
346          List<HRegion> onlineRegions = rsThread.getRegionServer().getRegions(LABELS_TABLE_NAME);
347          if (onlineRegions.size() > 0) {
348            break;
349          } else {
350            Thread.sleep(10);
351          }
352        } else {
353          break;
354        }
355      }
356    }
357    TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000);
358    t.join();
359    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
360      Scan s = new Scan();
361      s.setAuthorizations(new Authorizations(SECRET));
362      ResultScanner scanner = table.getScanner(s);
363      Result[] next = scanner.next(3);
364      assertTrue(next.length == 1);
365    }
366  }
367
368  @Test
369  public void testVisibilityLabelsOnRSRestart() throws Exception {
370    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
371    List<RegionServerThread> regionServerThreads =
372      TEST_UTIL.getHBaseCluster().getRegionServerThreads();
373    for (RegionServerThread rsThread : regionServerThreads) {
374      rsThread.getRegionServer().abort("Aborting ");
375    }
376    // Start one new RS
377    RegionServerThread rs = TEST_UTIL.getHBaseCluster().startRegionServer();
378    waitForLabelsRegionAvailability(rs.getRegionServer());
379    try (Table table = createTableAndWriteDataWithLabels(tableName,
380      "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE)) {
381      Scan s = new Scan();
382      s.setAuthorizations(new Authorizations(SECRET));
383      ResultScanner scanner = table.getScanner(s);
384      Result[] next = scanner.next(3);
385      assertTrue(next.length == 1);
386    }
387  }
388
389  protected void waitForLabelsRegionAvailability(HRegionServer regionServer) {
390    while (!regionServer.isOnline()) {
391      try {
392        Thread.sleep(10);
393      } catch (InterruptedException e) {
394      }
395    }
396    while (regionServer.getRegions(LABELS_TABLE_NAME).isEmpty()) {
397      try {
398        Thread.sleep(10);
399      } catch (InterruptedException e) {
400      }
401    }
402  }
403
404  @Test
405  public void testVisibilityLabelsInGetThatDoesNotMatchAnyDefinedLabels() throws Exception {
406    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
407    try (Table table = createTableAndWriteDataWithLabels(tableName,
408      "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE)) {
409      Get get = new Get(row1);
410      get.setAuthorizations(new Authorizations("SAMPLE"));
411      Result result = table.get(get);
412      assertTrue(result.isEmpty());
413    }
414  }
415
416  @Test
417  public void testSetAndGetUserAuths() throws Throwable {
418    final String user = "user1";
419    PrivilegedExceptionAction<Void> action = new PrivilegedExceptionAction<Void>() {
420      @Override
421      public Void run() throws Exception {
422        String[] auths = { SECRET, CONFIDENTIAL };
423        try (Connection conn = ConnectionFactory.createConnection(conf)) {
424          VisibilityClient.setAuths(conn, auths, user);
425        } catch (Throwable e) {
426          throw new IOException(e);
427        }
428        return null;
429      }
430    };
431    SUPERUSER.runAs(action);
432    try (Table ht = TEST_UTIL.getConnection().getTable(LABELS_TABLE_NAME)) {
433      Scan scan = new Scan();
434      scan.setAuthorizations(new Authorizations(VisibilityUtils.SYSTEM_LABEL));
435      ResultScanner scanner = ht.getScanner(scan);
436      Result result = null;
437      List<Result> results = new ArrayList<>();
438      while ((result = scanner.next()) != null) {
439        results.add(result);
440      }
441      List<String> auths = extractAuths(user, results);
442      assertTrue(auths.contains(SECRET));
443      assertTrue(auths.contains(CONFIDENTIAL));
444      assertEquals(2, auths.size());
445    }
446
447    action = new PrivilegedExceptionAction<Void>() {
448      @Override
449      public Void run() throws Exception {
450        GetAuthsResponse authsResponse = null;
451        try (Connection conn = ConnectionFactory.createConnection(conf)) {
452          authsResponse = VisibilityClient.getAuths(conn, user);
453        } catch (Throwable e) {
454          throw new IOException(e);
455        }
456        List<String> authsList = new ArrayList<>(authsResponse.getAuthList().size());
457        for (ByteString authBS : authsResponse.getAuthList()) {
458          authsList.add(Bytes.toString(authBS.toByteArray()));
459        }
460        assertEquals(2, authsList.size());
461        assertTrue(authsList.contains(SECRET));
462        assertTrue(authsList.contains(CONFIDENTIAL));
463        return null;
464      }
465    };
466    SUPERUSER.runAs(action);
467
468    // Try doing setAuths once again and there should not be any duplicates
469    action = new PrivilegedExceptionAction<Void>() {
470      @Override
471      public Void run() throws Exception {
472        String[] auths1 = { SECRET, CONFIDENTIAL };
473        GetAuthsResponse authsResponse = null;
474        try (Connection conn = ConnectionFactory.createConnection(conf)) {
475          VisibilityClient.setAuths(conn, auths1, user);
476          try {
477            authsResponse = VisibilityClient.getAuths(conn, user);
478          } catch (Throwable e) {
479            throw new IOException(e);
480          }
481        } catch (Throwable e) {
482        }
483        List<String> authsList = new ArrayList<>(authsResponse.getAuthList().size());
484        for (ByteString authBS : authsResponse.getAuthList()) {
485          authsList.add(Bytes.toString(authBS.toByteArray()));
486        }
487        assertEquals(2, authsList.size());
488        assertTrue(authsList.contains(SECRET));
489        assertTrue(authsList.contains(CONFIDENTIAL));
490        return null;
491      }
492    };
493    SUPERUSER.runAs(action);
494  }
495
496  protected List<String> extractAuths(String user, List<Result> results) {
497    List<String> auths = new ArrayList<>();
498    for (Result result : results) {
499      Cell labelCell = result.getColumnLatestCell(LABELS_TABLE_FAMILY, LABEL_QUALIFIER);
500      Cell userAuthCell = result.getColumnLatestCell(LABELS_TABLE_FAMILY, Bytes.toBytes(user));
501      if (userAuthCell != null) {
502        auths.add(Bytes.toString(labelCell.getValueArray(), labelCell.getValueOffset(),
503          labelCell.getValueLength()));
504      }
505    }
506    return auths;
507  }
508
509  @Test
510  public void testClearUserAuths() throws Throwable {
511    PrivilegedExceptionAction<Void> action = new PrivilegedExceptionAction<Void>() {
512      @Override
513      public Void run() throws Exception {
514        String[] auths = { SECRET, CONFIDENTIAL, PRIVATE };
515        String user = "testUser";
516        try (Connection conn = ConnectionFactory.createConnection(conf)) {
517          VisibilityClient.setAuths(conn, auths, user);
518        } catch (Throwable e) {
519          throw new IOException(e);
520        }
521        // Removing the auths for SECRET and CONFIDENTIAL for the user.
522        // Passing a non existing auth also.
523        auths = new String[] { SECRET, PUBLIC, CONFIDENTIAL };
524        VisibilityLabelsResponse response = null;
525        try (Connection conn = ConnectionFactory.createConnection(conf)) {
526          response = VisibilityClient.clearAuths(conn, auths, user);
527        } catch (Throwable e) {
528          fail("Should not have failed");
529        }
530        List<RegionActionResult> resultList = response.getResultList();
531        assertEquals(3, resultList.size());
532        assertTrue(resultList.get(0).getException().getValue().isEmpty());
533        assertEquals("org.apache.hadoop.hbase.DoNotRetryIOException",
534          resultList.get(1).getException().getName());
535        assertTrue(Bytes.toString(resultList.get(1).getException().getValue().toByteArray())
536          .contains("org.apache.hadoop.hbase.security.visibility.InvalidLabelException: "
537            + "Label 'public' is not set for the user testUser"));
538        assertTrue(resultList.get(2).getException().getValue().isEmpty());
539        try (Connection connection = ConnectionFactory.createConnection(conf);
540          Table ht = connection.getTable(LABELS_TABLE_NAME)) {
541          ResultScanner scanner = ht.getScanner(new Scan());
542          Result result = null;
543          List<Result> results = new ArrayList<>();
544          while ((result = scanner.next()) != null) {
545            results.add(result);
546          }
547          List<String> curAuths = extractAuths(user, results);
548          assertTrue(curAuths.contains(PRIVATE));
549          assertEquals(1, curAuths.size());
550        }
551
552        GetAuthsResponse authsResponse = null;
553        try (Connection conn = ConnectionFactory.createConnection(conf)) {
554          authsResponse = VisibilityClient.getAuths(conn, user);
555        } catch (Throwable e) {
556          throw new IOException(e);
557        }
558        List<String> authsList = new ArrayList<>(authsResponse.getAuthList().size());
559        for (ByteString authBS : authsResponse.getAuthList()) {
560          authsList.add(Bytes.toString(authBS.toByteArray()));
561        }
562        assertEquals(1, authsList.size());
563        assertTrue(authsList.contains(PRIVATE));
564        return null;
565      }
566    };
567    SUPERUSER.runAs(action);
568  }
569
570  @Test
571  public void testLabelsWithCheckAndPut() throws Throwable {
572    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
573    try (Table table = TEST_UTIL.createTable(tableName, fam)) {
574      byte[] row1 = Bytes.toBytes("row1");
575      Put put = new Put(row1);
576      put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, value);
577      put.setCellVisibility(new CellVisibility(SECRET + " & " + CONFIDENTIAL));
578      table.checkAndMutate(row1, fam).qualifier(qual).ifNotExists().thenPut(put);
579      byte[] row2 = Bytes.toBytes("row2");
580      put = new Put(row2);
581      put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, value);
582      put.setCellVisibility(new CellVisibility(SECRET));
583      table.checkAndMutate(row2, fam).qualifier(qual).ifNotExists().thenPut(put);
584
585      Scan scan = new Scan();
586      scan.setAuthorizations(new Authorizations(SECRET));
587      ResultScanner scanner = table.getScanner(scan);
588      Result result = scanner.next();
589      assertTrue(!result.isEmpty());
590      assertTrue(Bytes.equals(row2, result.getRow()));
591      result = scanner.next();
592      assertNull(result);
593    }
594  }
595
596  @Test
597  public void testLabelsWithIncrement() throws Throwable {
598    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
599    try (Table table = TEST_UTIL.createTable(tableName, fam)) {
600      byte[] row1 = Bytes.toBytes("row1");
601      byte[] val = Bytes.toBytes(1L);
602      Put put = new Put(row1);
603      put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, val);
604      put.setCellVisibility(new CellVisibility(SECRET + " & " + CONFIDENTIAL));
605      table.put(put);
606      Get get = new Get(row1);
607      get.setAuthorizations(new Authorizations(SECRET));
608      Result result = table.get(get);
609      assertTrue(result.isEmpty());
610      table.incrementColumnValue(row1, fam, qual, 2L);
611      result = table.get(get);
612      assertTrue(result.isEmpty());
613      Increment increment = new Increment(row1);
614      increment.addColumn(fam, qual, 2L);
615      increment.setCellVisibility(new CellVisibility(SECRET));
616      table.increment(increment);
617      result = table.get(get);
618      assertTrue(!result.isEmpty());
619    }
620  }
621
622  @Test
623  public void testLabelsWithAppend() throws Throwable {
624    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
625    try (Table table = TEST_UTIL.createTable(tableName, fam)) {
626      byte[] row1 = Bytes.toBytes("row1");
627      byte[] val = Bytes.toBytes("a");
628      Put put = new Put(row1);
629      put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, val);
630      put.setCellVisibility(new CellVisibility(SECRET + " & " + CONFIDENTIAL));
631      table.put(put);
632      Get get = new Get(row1);
633      get.setAuthorizations(new Authorizations(SECRET));
634      Result result = table.get(get);
635      assertTrue(result.isEmpty());
636      Append append = new Append(row1);
637      append.addColumn(fam, qual, Bytes.toBytes("b"));
638      table.append(append);
639      result = table.get(get);
640      assertTrue(result.isEmpty());
641      append = new Append(row1);
642      append.addColumn(fam, qual, Bytes.toBytes("c"));
643      append.setCellVisibility(new CellVisibility(SECRET));
644      table.append(append);
645      result = table.get(get);
646      assertTrue(!result.isEmpty());
647    }
648  }
649
650  @Test
651  public void testUserShouldNotDoDDLOpOnLabelsTable() throws Exception {
652    Admin admin = TEST_UTIL.getAdmin();
653    try {
654      admin.disableTable(LABELS_TABLE_NAME);
655      fail("Lables table should not get disabled by user.");
656    } catch (Exception e) {
657    }
658    try {
659      admin.deleteTable(LABELS_TABLE_NAME);
660      fail("Lables table should not get disabled by user.");
661    } catch (Exception e) {
662    }
663    try {
664      ColumnFamilyDescriptor columnFamilyDescriptor =
665        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("testFamily")).build();
666      admin.addColumnFamily(LABELS_TABLE_NAME, columnFamilyDescriptor);
667      fail("Lables table should not get altered by user.");
668    } catch (Exception e) {
669    }
670    try {
671      admin.deleteColumnFamily(LABELS_TABLE_NAME, VisibilityConstants.LABELS_TABLE_FAMILY);
672      fail("Lables table should not get altered by user.");
673    } catch (Exception e) {
674    }
675    try {
676      ColumnFamilyDescriptor familyDescriptor =
677        ColumnFamilyDescriptorBuilder.newBuilder(VisibilityConstants.LABELS_TABLE_FAMILY)
678          .setBloomFilterType(BloomType.ROWCOL).build();
679      admin.modifyColumnFamily(LABELS_TABLE_NAME, familyDescriptor);
680      fail("Lables table should not get altered by user.");
681    } catch (Exception e) {
682    }
683    try {
684      TableDescriptorBuilder tableDescriptorBuilder =
685        TableDescriptorBuilder.newBuilder(LABELS_TABLE_NAME);
686      ColumnFamilyDescriptor columnFamilyDescriptor =
687        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f1")).build();
688      tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
689      columnFamilyDescriptor =
690        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f2")).build();
691      tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
692      admin.modifyTable(tableDescriptorBuilder.build());
693      fail("Lables table should not get altered by user.");
694    } catch (Exception e) {
695    }
696  }
697
698  @Test
699  public void testMultipleVersions() throws Exception {
700    final byte[] r1 = Bytes.toBytes("row1");
701    final byte[] r2 = Bytes.toBytes("row2");
702    final byte[] v1 = Bytes.toBytes("100");
703    final byte[] v2 = Bytes.toBytes("101");
704    final byte[] fam2 = Bytes.toBytes("info2");
705    final byte[] qual2 = Bytes.toBytes("qual2");
706    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
707    // Default max versions is 1.
708    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
709      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam))
710      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam2).setMaxVersions(5).build())
711      .build();
712    TEST_UTIL.getAdmin().createTable(tableDescriptor);
713    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
714      Put put = new Put(r1);
715      put.addColumn(fam, qual, 3L, v1);
716      put.addColumn(fam, qual2, 3L, v1);
717      put.addColumn(fam2, qual, 3L, v1);
718      put.addColumn(fam2, qual2, 3L, v1);
719      put.setCellVisibility(new CellVisibility(SECRET));
720      table.put(put);
721      put = new Put(r1);
722      put.addColumn(fam, qual, 4L, v2);
723      put.addColumn(fam, qual2, 4L, v2);
724      put.addColumn(fam2, qual, 4L, v2);
725      put.addColumn(fam2, qual2, 4L, v2);
726      put.setCellVisibility(new CellVisibility(PRIVATE));
727      table.put(put);
728
729      put = new Put(r2);
730      put.addColumn(fam, qual, 3L, v1);
731      put.addColumn(fam, qual2, 3L, v1);
732      put.addColumn(fam2, qual, 3L, v1);
733      put.addColumn(fam2, qual2, 3L, v1);
734      put.setCellVisibility(new CellVisibility(SECRET));
735      table.put(put);
736      put = new Put(r2);
737      put.addColumn(fam, qual, 4L, v2);
738      put.addColumn(fam, qual2, 4L, v2);
739      put.addColumn(fam2, qual, 4L, v2);
740      put.addColumn(fam2, qual2, 4L, v2);
741      put.setCellVisibility(new CellVisibility(SECRET));
742      table.put(put);
743
744      Scan s = new Scan();
745      s.readVersions(1);
746      s.setAuthorizations(new Authorizations(SECRET));
747      ResultScanner scanner = table.getScanner(s);
748      Result result = scanner.next();
749      assertTrue(Bytes.equals(r1, result.getRow()));
750      // for cf 'fam' max versions in HCD is 1. So the old version cells, which are having matching
751      // CellVisibility with Authorizations, should not get considered in the label evaluation at
752      // all.
753      assertNull(result.getColumnLatestCell(fam, qual));
754      assertNull(result.getColumnLatestCell(fam, qual2));
755      // for cf 'fam2' max versions in HCD is > 1. So we can consider the old version cells, which
756      // are having matching CellVisibility with Authorizations, in the label evaluation. It can
757      // just skip those recent versions for which visibility is not there as per the new version's
758      // CellVisibility. The old versions which are having visibility can be send back
759      Cell cell = result.getColumnLatestCell(fam2, qual);
760      assertNotNull(cell);
761      assertTrue(Bytes.equals(v1, 0, v1.length, cell.getValueArray(), cell.getValueOffset(),
762        cell.getValueLength()));
763      cell = result.getColumnLatestCell(fam2, qual2);
764      assertNotNull(cell);
765      assertTrue(Bytes.equals(v1, 0, v1.length, cell.getValueArray(), cell.getValueOffset(),
766        cell.getValueLength()));
767
768      result = scanner.next();
769      assertTrue(Bytes.equals(r2, result.getRow()));
770      cell = result.getColumnLatestCell(fam, qual);
771      assertNotNull(cell);
772      assertTrue(Bytes.equals(v2, 0, v2.length, cell.getValueArray(), cell.getValueOffset(),
773        cell.getValueLength()));
774      cell = result.getColumnLatestCell(fam, qual2);
775      assertNotNull(cell);
776      assertTrue(Bytes.equals(v2, 0, v2.length, cell.getValueArray(), cell.getValueOffset(),
777        cell.getValueLength()));
778      cell = result.getColumnLatestCell(fam2, qual);
779      assertNotNull(cell);
780      assertTrue(Bytes.equals(v2, 0, v2.length, cell.getValueArray(), cell.getValueOffset(),
781        cell.getValueLength()));
782      cell = result.getColumnLatestCell(fam2, qual2);
783      assertNotNull(cell);
784      assertTrue(Bytes.equals(v2, 0, v2.length, cell.getValueArray(), cell.getValueOffset(),
785        cell.getValueLength()));
786    }
787  }
788
789  @Test
790  public void testMutateRow() throws Exception {
791    final byte[] qual2 = Bytes.toBytes("qual2");
792    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
793    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
794      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build();
795    TEST_UTIL.getAdmin().createTable(tableDescriptor);
796    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
797      Put p1 = new Put(row1);
798      p1.addColumn(fam, qual, value);
799      p1.setCellVisibility(new CellVisibility(CONFIDENTIAL));
800
801      Put p2 = new Put(row1);
802      p2.addColumn(fam, qual2, value);
803      p2.setCellVisibility(new CellVisibility(SECRET));
804
805      RowMutations rm = new RowMutations(row1);
806      rm.add(p1);
807      rm.add(p2);
808
809      table.mutateRow(rm);
810
811      Get get = new Get(row1);
812      get.setAuthorizations(new Authorizations(CONFIDENTIAL));
813      Result result = table.get(get);
814      assertTrue(result.containsColumn(fam, qual));
815      assertFalse(result.containsColumn(fam, qual2));
816
817      get.setAuthorizations(new Authorizations(SECRET));
818      result = table.get(get);
819      assertFalse(result.containsColumn(fam, qual));
820      assertTrue(result.containsColumn(fam, qual2));
821    }
822  }
823
824  @Test
825  public void testFlushedFileWithVisibilityTags() throws Exception {
826    final byte[] qual2 = Bytes.toBytes("qual2");
827    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
828    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
829      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build();
830    TEST_UTIL.getAdmin().createTable(tableDescriptor);
831    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
832      Put p1 = new Put(row1);
833      p1.addColumn(fam, qual, value);
834      p1.setCellVisibility(new CellVisibility(CONFIDENTIAL));
835
836      Put p2 = new Put(row1);
837      p2.addColumn(fam, qual2, value);
838      p2.setCellVisibility(new CellVisibility(SECRET));
839
840      RowMutations rm = new RowMutations(row1);
841      rm.add(p1);
842      rm.add(p2);
843
844      table.mutateRow(rm);
845    }
846    TEST_UTIL.getAdmin().flush(tableName);
847    List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
848    HStore store = regions.get(0).getStore(fam);
849    Collection<HStoreFile> storefiles = store.getStorefiles();
850    assertTrue(storefiles.size() > 0);
851    for (HStoreFile storeFile : storefiles) {
852      assertTrue(storeFile.getReader().getHFileReader().getFileContext().isIncludesTags());
853    }
854  }
855
856  static Table createTableAndWriteDataWithLabels(TableName tableName, String... labelExps)
857    throws Exception {
858    List<Put> puts = new ArrayList<>(labelExps.length);
859    for (int i = 0; i < labelExps.length; i++) {
860      Put put = new Put(Bytes.toBytes("row" + (i + 1)));
861      put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, value);
862      put.setCellVisibility(new CellVisibility(labelExps[i]));
863      puts.add(put);
864    }
865    Table table = TEST_UTIL.createTable(tableName, fam);
866    table.put(puts);
867    return table;
868  }
869
870  public static void addLabels() throws Exception {
871    PrivilegedExceptionAction<VisibilityLabelsResponse> action =
872      new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
873        @Override
874        public VisibilityLabelsResponse run() throws Exception {
875          String[] labels = { SECRET, TOPSECRET, CONFIDENTIAL, PUBLIC, PRIVATE, COPYRIGHT, ACCENT,
876            UNICODE_VIS_TAG, UC1, UC2 };
877          try (Connection conn = ConnectionFactory.createConnection(conf)) {
878            VisibilityClient.addLabels(conn, labels);
879          } catch (Throwable t) {
880            throw new IOException(t);
881          }
882          return null;
883        }
884      };
885    SUPERUSER.runAs(action);
886  }
887}