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.access;
20  
21  import static org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint.Consumer;
22  import static org.junit.Assert.assertEquals;
23  
24  import com.google.common.collect.Multimap;
25  import java.io.IOException;
26  import java.nio.ByteBuffer;
27  import java.util.Arrays;
28  import java.util.Deque;
29  import java.util.HashSet;
30  import java.util.Set;
31  import java.util.concurrent.ExecutorService;
32  import java.util.concurrent.atomic.AtomicReference;
33  
34  import org.apache.commons.lang.StringUtils;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.Cell;
38  import org.apache.hadoop.hbase.DoNotRetryIOException;
39  import org.apache.hadoop.hbase.HBaseConfiguration;
40  import org.apache.hadoop.hbase.HBaseTestingUtility;
41  import org.apache.hadoop.hbase.HColumnDescriptor;
42  import org.apache.hadoop.hbase.HTableDescriptor;
43  import org.apache.hadoop.hbase.TableName;
44  import org.apache.hadoop.hbase.client.Connection;
45  import org.apache.hadoop.hbase.client.Get;
46  import org.apache.hadoop.hbase.client.Put;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.client.Table;
49  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
50  import org.apache.hadoop.hbase.io.compress.Compression;
51  import org.apache.hadoop.hbase.io.crypto.Encryption;
52  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
53  import org.apache.hadoop.hbase.io.hfile.HFile;
54  import org.apache.hadoop.hbase.io.hfile.HFileContext;
55  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
56  import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
57  import org.apache.hadoop.hbase.regionserver.HStore;
58  import org.apache.hadoop.hbase.regionserver.Region;
59  import org.apache.hadoop.hbase.regionserver.StoreFile;
60  import org.apache.hadoop.hbase.testclassification.MediumTests;
61  import org.apache.hadoop.hbase.util.Bytes;
62  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
63  import org.apache.hadoop.hbase.util.Threads;
64  import org.junit.AfterClass;
65  import org.junit.Assert;
66  import org.junit.BeforeClass;
67  import org.junit.Test;
68  import org.junit.experimental.categories.Category;
69  import org.slf4j.Logger;
70  import org.slf4j.LoggerFactory;
71  
72  /**
73   * Tests the SecureBulkLoadEndpoint code.
74   */
75  @Category(MediumTests.class)
76  public class TestSecureBulkLoadEndpoint {
77  
78    private static final Logger LOG =
79        LoggerFactory.getLogger(TestSecureBulkLoadEndpoint.class);
80  
81    private static TableName TABLE = TableName.valueOf(Bytes.toBytes("TestSecureBulkLoadManager"));
82    private static byte[] FAMILY = Bytes.toBytes("family");
83    private static byte[] COLUMN = Bytes.toBytes("column");
84    private static byte[] key1 = Bytes.toBytes("row1");
85    private static byte[] key2 = Bytes.toBytes("row2");
86    private static byte[] key3 = Bytes.toBytes("row3");
87    private static byte[] value1 = Bytes.toBytes("t1");
88    private static byte[] value3 = Bytes.toBytes("t3");
89    private static byte[] SPLIT_ROWKEY = key2;
90  
91    private Thread ealierBulkload;
92    private Thread laterBulkload;
93  
94    protected final static HBaseTestingUtility testUtil = new HBaseTestingUtility();
95    private static Configuration conf = testUtil.getConfiguration();
96  
97    @BeforeClass
98    public static void setUp() throws Exception {
99      conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,SecureBulkLoadEndpoint.class.getName());
100     testUtil.startMiniCluster();
101   }
102 
103   @AfterClass
104   public static void tearDown() throws Exception {
105     testUtil.shutdownMiniCluster();
106     testUtil.cleanupTestDir();
107   }
108 
109   @Test
110   public void testFileSystemsWithoutPermissionSupport() {
111     final Configuration emptyConf = new Configuration(false);
112     final Configuration defaultConf = HBaseConfiguration.create();
113 
114     final Set<String> expectedDefaultIgnoredSchemes = new HashSet<>(
115         Arrays.asList(
116           StringUtils.split(SecureBulkLoadEndpoint.FS_WITHOUT_SUPPORT_PERMISSION_DEFAULT, ',')));
117 
118     final SecureBulkLoadEndpoint endpoint = new SecureBulkLoadEndpoint();
119 
120     // Empty configuration should return the default list of schemes
121     Set<String> defaultIgnoredSchemes = endpoint.getFileSystemSchemesWithoutPermissionSupport(
122         emptyConf);
123     assertEquals(defaultIgnoredSchemes, expectedDefaultIgnoredSchemes);
124 
125     // Default configuration (unset) should be the default list of schemes
126     defaultIgnoredSchemes = endpoint.getFileSystemSchemesWithoutPermissionSupport(defaultConf);
127     assertEquals(defaultIgnoredSchemes, expectedDefaultIgnoredSchemes);
128 
129     defaultConf.set(SecureBulkLoadEndpoint.FS_WITHOUT_SUPPORT_PERMISSION_KEY, "foo,bar");
130     defaultIgnoredSchemes = endpoint.getFileSystemSchemesWithoutPermissionSupport(defaultConf);
131     assertEquals(defaultIgnoredSchemes, new HashSet<String>(Arrays.asList("foo", "bar")));
132   }
133 
134   /**
135    * After a secure bulkload finished , there is a clean-up for FileSystems used in the bulkload.
136    * Sometimes, FileSystems used in the finished bulkload might also be used in other bulkload
137    * calls, or there are other FileSystems created by the same user, they could be closed by a
138    * FileSystem.closeAllForUGI call. So during the clean-up, those FileSystems need to be used
139    * later can not get closed ,or else a race condition occurs.
140    *
141    * testForRaceCondition tests the case that two secure bulkload calls from the same UGI go
142    * into two different regions and one bulkload finishes earlier when the other bulkload still
143    * needs its FileSystems, checks that both bulkloads succeed.
144    */
145   @Test
146   public void testForRaceCondition() throws Exception {
147     /// create table
148     testUtil.createTable(TABLE,FAMILY,Bytes.toByteArrays(SPLIT_ROWKEY));
149     testUtil.waitUntilAllRegionsAssigned(TABLE);
150 
151     Consumer<Region> fsCreatedListener = new Consumer<Region>() {
152       @Override
153       public void accept(Region hRegion) {
154         if (hRegion.getRegionInfo().containsRow(key3)) {
155           Threads.shutdown(ealierBulkload);/// wait util the other bulkload finished
156         }
157       }
158     };
159     SecureBulkLoadEndpoint.setFsCreatedListener(fsCreatedListener);
160 
161     /// prepare files
162     Path rootdir = testUtil.getMiniHBaseCluster().getRegionServerThreads().get(0)
163         .getRegionServer().getFileSystem().getHomeDirectory();
164     final Path dir1 = new Path(rootdir, "dir1");
165     prepareHFile(dir1, key1, value1);
166     final Path dir2 = new Path(rootdir, "dir2");
167     prepareHFile(dir2, key3, value3);
168 
169     /// do bulkload
170     final AtomicReference<Throwable> t1Exception = new AtomicReference<>();
171     final AtomicReference<Throwable> t2Exception = new AtomicReference<>();
172     ealierBulkload = new Thread(new Runnable() {
173       @Override
174       public void run() {
175         try {
176           doBulkloadWithoutRetry(dir1);
177         } catch (Exception e) {
178           LOG.error("bulk load failed .",e);
179           t1Exception.set(e);
180         }
181       }
182     });
183     laterBulkload = new Thread(new Runnable() {
184       @Override
185       public void run() {
186         try {
187           doBulkloadWithoutRetry(dir2);
188         } catch (Exception e) {
189           LOG.error("bulk load failed .",e);
190           t2Exception.set(e);
191         }
192       }
193     });
194     ealierBulkload.start();
195     laterBulkload.start();
196     Threads.shutdown(ealierBulkload);
197     Threads.shutdown(laterBulkload);
198     Assert.assertNull(t1Exception.get());
199     Assert.assertNull(t2Exception.get());
200 
201     /// check bulkload ok
202     Get get1 = new Get(key1);
203     Get get3 = new Get(key3);
204     Table t = testUtil.getConnection().getTable(TABLE);
205     Result r = t.get(get1);
206     Assert.assertArrayEquals(r.getValue(FAMILY, COLUMN), value1);
207     r = t.get(get3);
208     Assert.assertArrayEquals(r.getValue(FAMILY, COLUMN), value3);
209 
210   }
211 
212   /**
213    * A trick is used to make sure server-side failures( if any ) not being covered up by a client
214    * retry. Since LoadIncrementalHFiles.doBulkLoad keeps performing bulkload calls as long as the
215    * HFile queue is not empty, while server-side exceptions in the doAs block do not lead
216    * to a client exception, a bulkload will always succeed in this case by default, thus client
217    * will never be aware that failures have ever happened . To avoid this kind of retry ,
218    * a MyExceptionToAvoidRetry exception is thrown after bulkLoadPhase finished and caught
219    * silently outside the doBulkLoad call, so that the bulkLoadPhase would be called exactly
220    * once, and server-side failures, if any ,can be checked via data.
221    */
222   class MyExceptionToAvoidRetry extends DoNotRetryIOException {
223   }
224 
225   private void doBulkloadWithoutRetry(Path dir) throws Exception {
226     Connection connection = testUtil.getConnection();
227     LoadIncrementalHFiles h = new LoadIncrementalHFiles(conf) {
228       @Override
229       protected void bulkLoadPhase(final Table table, final Connection conn,
230           ExecutorService pool, Deque<LoadQueueItem> queue,
231           final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
232         super.bulkLoadPhase(table, conn, pool, queue, regionGroups);
233         throw new MyExceptionToAvoidRetry(); // throw exception to avoid retry
234       }
235     };
236     try {
237       h.doBulkLoad(dir, testUtil.getHBaseAdmin(), connection.getTable(TABLE),
238           connection.getRegionLocator(TABLE));
239       Assert.fail("MyExceptionToAvoidRetry is expected");
240     } catch (MyExceptionToAvoidRetry e) { //expected
241     }
242   }
243 
244   private void prepareHFile(Path dir, byte[] key, byte[] value) throws Exception {
245     HTableDescriptor desc = testUtil.getHBaseAdmin().getTableDescriptor(TABLE);
246     HColumnDescriptor family = desc.getFamily(FAMILY);
247     Compression.Algorithm compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
248 
249     CacheConfig writerCacheConf = new CacheConfig(conf, family);
250     writerCacheConf.setCacheDataOnWrite(false);
251     HFileContext hFileContext = new HFileContextBuilder()
252         .withIncludesMvcc(false)
253         .withIncludesTags(true)
254         .withCompression(compression)
255         .withCompressTags(family.isCompressTags())
256         .withChecksumType(HStore.getChecksumType(conf))
257         .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
258         .withBlockSize(family.getBlocksize())
259         .withHBaseCheckSum(true)
260         .withDataBlockEncoding(family.getDataBlockEncoding())
261         .withEncryptionContext(Encryption.Context.NONE)
262         .withCreateTime(EnvironmentEdgeManager.currentTime())
263         .build();
264     StoreFile.WriterBuilder builder =
265         new StoreFile.WriterBuilder(conf, writerCacheConf, dir.getFileSystem(conf))
266             .withOutputDir(new Path(dir, family.getNameAsString()))
267             .withBloomType(family.getBloomFilterType())
268             .withMaxKeyCount(Integer.MAX_VALUE)
269             .withFileContext(hFileContext);
270     StoreFile.Writer writer = builder.build();
271 
272     Put put = new Put(key);
273     put.addColumn(FAMILY, COLUMN, value);
274     for (Cell c : put.get(FAMILY, COLUMN)) {
275       writer.append(c);
276     }
277 
278     writer.close();
279   }
280 
281 }