1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17 package org.apache.hadoop.hbase.quotas.policies;
18
19 import static org.mockito.Mockito.mock;
20 import static org.mockito.Mockito.when;
21
22 import java.util.ArrayList;
23 import java.util.List;
24
25 import org.apache.hadoop.fs.FileStatus;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.TableName;
29 import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
30 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
31 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
32 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
33 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
34 import org.apache.hadoop.hbase.testclassification.SmallTests;
35 import org.junit.Before;
36 import org.junit.Test;
37 import org.junit.experimental.categories.Category;
38
39
40
41
42 @Category(SmallTests.class)
43 public class TestBulkLoadCheckingViolationPolicyEnforcement {
44
45 FileSystem fs;
46 RegionServerServices rss;
47 TableName tableName;
48 SpaceViolationPolicyEnforcement policy;
49
50 @Before
51 public void setup() {
52 fs = mock(FileSystem.class);
53 rss = mock(RegionServerServices.class);
54 tableName = TableName.valueOf("foo");
55 policy = new DefaultViolationPolicyEnforcement();
56 }
57
58 @Test
59 public void testFilesUnderLimit() throws Exception {
60 final List<String> paths = new ArrayList<>();
61 final List<FileStatus> statuses = new ArrayList<>();
62 final long length = 100L * 1024L;
63 for (int i = 0; i < 5; i++) {
64 String path = "/" + i;
65 FileStatus status = mock(FileStatus.class);
66 when(fs.getFileStatus(new Path(path))).thenReturn(status);
67 when(status.getLen()).thenReturn(length);
68 when(status.isFile()).thenReturn(true);
69 paths.add(path);
70 statuses.add(status);
71 }
72
73
74 SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, length * 6);
75
76 policy.initialize(rss, tableName, snapshot);
77
78 policy.checkBulkLoad(fs, paths);
79 }
80
81 @Test(expected = IllegalArgumentException.class)
82 public void testFileIsNotAFile() throws Exception {
83 final List<String> paths = new ArrayList<>();
84 String path = "/1";
85 FileStatus status = mock(FileStatus.class);
86 when(fs.getFileStatus(new Path(path))).thenReturn(status);
87 when(status.getLen()).thenReturn(1000L);
88 when(status.isFile()).thenReturn(false);
89 paths.add(path);
90
91
92 SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, Long.MAX_VALUE);
93
94 policy.initialize(rss, tableName, snapshot);
95
96
97 policy.checkBulkLoad(fs, paths);
98 }
99
100 @Test(expected = SpaceLimitingException.class)
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
116 SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 1024L);
117
118 policy.initialize(rss, tableName, snapshot);
119
120 policy.checkBulkLoad(fs, paths);
121 }
122
123 @Test(expected = SpaceLimitingException.class)
124 public void testSumOfFilesOverLimit() throws Exception {
125 final List<String> paths = new ArrayList<>();
126 final List<FileStatus> statuses = new ArrayList<>();
127 final long length = 1024L;
128 for (int i = 0; i < 5; i++) {
129 String path = "/" + i;
130 FileStatus status = mock(FileStatus.class);
131 when(fs.getFileStatus(new Path(path))).thenReturn(status);
132 when(status.getLen()).thenReturn(length);
133 when(status.isFile()).thenReturn(true);
134 paths.add(path);
135 statuses.add(status);
136 }
137
138
139 SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 5000L);
140
141 policy.initialize(rss, tableName, snapshot);
142
143 policy.checkBulkLoad(fs, paths);
144 }
145
146 }