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;
019
020import static org.apache.hbase.thirdparty.com.google.common.collect.Iterables.size;
021import static org.junit.Assert.assertEquals;
022import static org.junit.Assert.assertNull;
023import static org.mockito.Matchers.any;
024import static org.mockito.Mockito.mock;
025import static org.mockito.Mockito.when;
026
027import java.io.IOException;
028import java.util.Collections;
029import java.util.HashMap;
030import java.util.Map;
031import java.util.concurrent.atomic.AtomicReference;
032import org.apache.hadoop.hbase.HBaseClassTestRule;
033import org.apache.hadoop.hbase.TableName;
034import org.apache.hadoop.hbase.client.Connection;
035import org.apache.hadoop.hbase.client.RegionInfo;
036import org.apache.hadoop.hbase.client.RegionInfoBuilder;
037import org.apache.hadoop.hbase.client.Result;
038import org.apache.hadoop.hbase.client.ResultScanner;
039import org.apache.hadoop.hbase.client.Scan;
040import org.apache.hadoop.hbase.client.Table;
041import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
042import org.apache.hadoop.hbase.testclassification.SmallTests;
043import org.apache.hadoop.hbase.util.Bytes;
044import org.junit.Before;
045import org.junit.ClassRule;
046import org.junit.Test;
047import org.junit.experimental.categories.Category;
048import org.mockito.invocation.InvocationOnMock;
049import org.mockito.stubbing.Answer;
050
051import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
052import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
053import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
054import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
055
056/**
057 * Test class for {@link TableQuotaSnapshotStore}.
058 */
059@Category(SmallTests.class)
060public class TestTableQuotaViolationStore {
061
062  @ClassRule
063  public static final HBaseClassTestRule CLASS_RULE =
064    HBaseClassTestRule.forClass(TestTableQuotaViolationStore.class);
065
066  private static final long ONE_MEGABYTE = 1024L * 1024L;
067
068  private Connection conn;
069  private QuotaObserverChore chore;
070  private Map<RegionInfo, Long> regionReports;
071  private TableQuotaSnapshotStore store;
072
073  @Before
074  public void setup() {
075    conn = mock(Connection.class);
076    chore = mock(QuotaObserverChore.class);
077    regionReports = new HashMap<>();
078    store = new TableQuotaSnapshotStore(conn, chore, regionReports);
079  }
080
081  @Test
082  public void testFilterRegionsByTable() throws Exception {
083    TableName tn1 = TableName.valueOf("foo");
084    TableName tn2 = TableName.valueOf("bar");
085    TableName tn3 = TableName.valueOf("ns", "foo");
086
087    assertEquals(0, size(store.filterBySubject(tn1)));
088
089    for (int i = 0; i < 5; i++) {
090      regionReports.put(RegionInfoBuilder.newBuilder(tn1).setStartKey(Bytes.toBytes(i))
091        .setEndKey(Bytes.toBytes(i + 1)).build(), 0L);
092    }
093    for (int i = 0; i < 3; i++) {
094      regionReports.put(RegionInfoBuilder.newBuilder(tn2).setStartKey(Bytes.toBytes(i))
095        .setEndKey(Bytes.toBytes(i + 1)).build(), 0L);
096    }
097    for (int i = 0; i < 10; i++) {
098      regionReports.put(RegionInfoBuilder.newBuilder(tn3).setStartKey(Bytes.toBytes(i))
099        .setEndKey(Bytes.toBytes(i + 1)).build(), 0L);
100    }
101    assertEquals(18, regionReports.size());
102    assertEquals(5, size(store.filterBySubject(tn1)));
103    assertEquals(3, size(store.filterBySubject(tn2)));
104    assertEquals(10, size(store.filterBySubject(tn3)));
105  }
106
107  @Test
108  public void testTargetViolationState() throws IOException {
109    mockNoSnapshotSizes();
110    TableName tn1 = TableName.valueOf("violation1");
111    TableName tn2 = TableName.valueOf("observance1");
112    TableName tn3 = TableName.valueOf("observance2");
113    SpaceQuota quota = SpaceQuota.newBuilder().setSoftLimit(1024L * 1024L)
114      .setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(SpaceViolationPolicy.DISABLE))
115      .build();
116
117    // Create some junk data to filter. Makes sure it's so large that it would
118    // immediately violate the quota.
119    for (int i = 0; i < 3; i++) {
120      regionReports.put(RegionInfoBuilder.newBuilder(tn2).setStartKey(Bytes.toBytes(i))
121        .setEndKey(Bytes.toBytes(i + 1)).build(), 5L * ONE_MEGABYTE);
122      regionReports.put(RegionInfoBuilder.newBuilder(tn3).setStartKey(Bytes.toBytes(i))
123        .setEndKey(Bytes.toBytes(i + 1)).build(), 5L * ONE_MEGABYTE);
124    }
125    regionReports.put(RegionInfoBuilder.newBuilder(tn1).setStartKey(Bytes.toBytes(0))
126      .setEndKey(Bytes.toBytes(1)).build(), 1024L * 512L);
127    regionReports.put(RegionInfoBuilder.newBuilder(tn1).setStartKey(Bytes.toBytes(1))
128      .setEndKey(Bytes.toBytes(2)).build(), 1024L * 256L);
129
130    SpaceQuotaSnapshot tn1Snapshot =
131      new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 1024L * 768L, 1024L * 1024L);
132
133    // Below the quota
134    assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
135
136    regionReports.put(RegionInfoBuilder.newBuilder(tn1).setStartKey(Bytes.toBytes(2))
137      .setEndKey(Bytes.toBytes(3)).build(), 1024L * 256L);
138    tn1Snapshot =
139      new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 1024L * 1024L, 1024L * 1024L);
140
141    // Equal to the quota is still in observance
142    assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
143
144    regionReports.put(RegionInfoBuilder.newBuilder(tn1).setStartKey(Bytes.toBytes(3))
145      .setEndKey(Bytes.toBytes(4)).build(), 1024L);
146    tn1Snapshot = new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE),
147      1024L * 1024L + 1024L, 1024L * 1024L);
148
149    // Exceeds the quota, should be in violation
150    assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
151  }
152
153  @Test
154  public void testGetSpaceQuota() throws Exception {
155    TableQuotaSnapshotStore mockStore = mock(TableQuotaSnapshotStore.class);
156    when(mockStore.getSpaceQuota(any())).thenCallRealMethod();
157
158    Quotas quotaWithSpace = Quotas.newBuilder().setSpace(SpaceQuota.newBuilder().setSoftLimit(1024L)
159      .setViolationPolicy(QuotaProtos.SpaceViolationPolicy.DISABLE).build()).build();
160    Quotas quotaWithoutSpace = Quotas.newBuilder().build();
161
162    AtomicReference<Quotas> quotaRef = new AtomicReference<>();
163    when(mockStore.getQuotaForTable(any())).then(new Answer<Quotas>() {
164      @Override
165      public Quotas answer(InvocationOnMock invocation) throws Throwable {
166        return quotaRef.get();
167      }
168    });
169
170    quotaRef.set(quotaWithSpace);
171    assertEquals(quotaWithSpace.getSpace(), mockStore.getSpaceQuota(TableName.valueOf("foo")));
172    quotaRef.set(quotaWithoutSpace);
173    assertNull(mockStore.getSpaceQuota(TableName.valueOf("foo")));
174  }
175
176  void mockNoSnapshotSizes() throws IOException {
177    Table quotaTable = mock(Table.class);
178    ResultScanner scanner = mock(ResultScanner.class);
179    when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
180    when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
181    when(scanner.iterator()).thenReturn(Collections.<Result> emptyList().iterator());
182  }
183}