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.quotas.policies; 019 020import static org.junit.jupiter.api.Assertions.assertThrows; 021import static org.mockito.Mockito.mock; 022import static org.mockito.Mockito.when; 023 024import java.util.ArrayList; 025import java.util.List; 026import org.apache.hadoop.fs.FileStatus; 027import org.apache.hadoop.fs.FileSystem; 028import org.apache.hadoop.fs.Path; 029import org.apache.hadoop.hbase.TableName; 030import org.apache.hadoop.hbase.quotas.SpaceLimitingException; 031import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; 032import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; 033import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement; 034import org.apache.hadoop.hbase.regionserver.RegionServerServices; 035import org.apache.hadoop.hbase.testclassification.SmallTests; 036import org.junit.jupiter.api.BeforeEach; 037import org.junit.jupiter.api.Tag; 038import org.junit.jupiter.api.Test; 039 040@Tag(SmallTests.TAG) 041public class TestBulkLoadCheckingViolationPolicyEnforcement { 042 043 FileSystem fs; 044 RegionServerServices rss; 045 TableName tableName; 046 SpaceViolationPolicyEnforcement policy; 047 048 @BeforeEach 049 public void setup() { 050 fs = mock(FileSystem.class); 051 rss = mock(RegionServerServices.class); 052 tableName = TableName.valueOf("foo"); 053 policy = new DefaultViolationPolicyEnforcement(); 054 } 055 056 @Test 057 public void testFilesUnderLimit() throws Exception { 058 final List<String> paths = new ArrayList<>(); 059 final List<FileStatus> statuses = new ArrayList<>(); 060 final long length = 100L * 1024L; 061 for (int i = 0; i < 5; i++) { 062 String path = "/" + i; 063 FileStatus status = mock(FileStatus.class); 064 when(fs.getFileStatus(new Path(path))).thenReturn(status); 065 when(status.getLen()).thenReturn(length); 066 when(status.isFile()).thenReturn(true); 067 paths.add(path); 068 statuses.add(status); 069 } 070 071 // Quota is not in violation now 072 SpaceQuotaSnapshot snapshot = 073 new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, length * 6); 074 075 policy.initialize(rss, tableName, snapshot); 076 077 policy.computeBulkLoadSize(fs, paths); 078 } 079 080 @Test 081 public void testFileIsNotAFile() throws Exception { 082 final List<String> paths = new ArrayList<>(); 083 String path = "/1"; 084 FileStatus status = mock(FileStatus.class); 085 when(fs.getFileStatus(new Path(path))).thenReturn(status); 086 when(status.getLen()).thenReturn(1000L); 087 when(status.isFile()).thenReturn(false); 088 paths.add(path); 089 090 // Quota is not in violation now 091 SpaceQuotaSnapshot snapshot = 092 new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, Long.MAX_VALUE); 093 094 policy.initialize(rss, tableName, snapshot); 095 096 // If the file to bulk load isn't a file, this should throw an exception 097 assertThrows(IllegalArgumentException.class, () -> policy.computeBulkLoadSize(fs, paths)); 098 } 099 100 @Test 101 public void testOneFileInBatchOverLimit() throws Exception { 102 final List<String> paths = new ArrayList<>(); 103 final List<FileStatus> statuses = new ArrayList<>(); 104 final long length = 1000L * 1024L; 105 for (int i = 0; i < 5; i++) { 106 String path = "/" + i; 107 FileStatus status = mock(FileStatus.class); 108 when(fs.getFileStatus(new Path(path))).thenReturn(status); 109 when(status.getLen()).thenReturn(length); 110 when(status.isFile()).thenReturn(true); 111 paths.add(path); 112 statuses.add(status); 113 } 114 115 // Quota is not in violation now 116 SpaceQuotaSnapshot snapshot = 117 new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 1024L); 118 119 policy.initialize(rss, tableName, snapshot); 120 121 assertThrows(SpaceLimitingException.class, () -> policy.computeBulkLoadSize(fs, paths)); 122 } 123 124 @Test 125 public void testSumOfFilesOverLimit() throws Exception { 126 final List<String> paths = new ArrayList<>(); 127 final List<FileStatus> statuses = new ArrayList<>(); 128 final long length = 1024L; 129 for (int i = 0; i < 5; i++) { 130 String path = "/" + i; 131 FileStatus status = mock(FileStatus.class); 132 when(fs.getFileStatus(new Path(path))).thenReturn(status); 133 when(status.getLen()).thenReturn(length); 134 when(status.isFile()).thenReturn(true); 135 paths.add(path); 136 statuses.add(status); 137 } 138 139 // Quota is not in violation now, but 5*1024 files would push us to violation 140 SpaceQuotaSnapshot snapshot = 141 new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 5000L); 142 143 policy.initialize(rss, tableName, snapshot); 144 145 assertThrows(SpaceLimitingException.class, () -> policy.computeBulkLoadSize(fs, paths)); 146 } 147}