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  package org.apache.hadoop.hbase.regionserver.compactions;
19  
20  import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY;
21  import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY;
22  import static org.junit.Assert.assertArrayEquals;
23  import static org.junit.Assert.assertEquals;
24  import static org.junit.Assert.assertFalse;
25  import static org.junit.Assert.assertNotNull;
26  import static org.junit.Assert.assertNull;
27  import static org.junit.Assert.assertTrue;
28  import static org.mockito.Matchers.any;
29  import static org.mockito.Matchers.anyBoolean;
30  import static org.mockito.Matchers.anyLong;
31  import static org.mockito.Mockito.doAnswer;
32  import static org.mockito.Mockito.mock;
33  import static org.mockito.Mockito.when;
34  
35  import java.io.IOException;
36  import java.util.ArrayList;
37  import java.util.Arrays;
38  import java.util.List;
39  import java.util.TreeMap;
40  
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.Cell;
43  import org.apache.hadoop.hbase.KeyValue;
44  import org.apache.hadoop.hbase.io.hfile.HFile;
45  import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter;
46  import org.apache.hadoop.hbase.regionserver.BloomType;
47  import org.apache.hadoop.hbase.regionserver.InternalScanner;
48  import org.apache.hadoop.hbase.regionserver.StoreFile;
49  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
50  import org.apache.hadoop.hbase.util.Bytes;
51  import org.mockito.invocation.InvocationOnMock;
52  import org.mockito.stubbing.Answer;
53  
54  public class TestCompactor {
55  
56    public static StoreFile createDummyStoreFile(long maxSequenceId) throws Exception {
57      // "Files" are totally unused, it's Scanner class below that gives compactor fake KVs.
58      // But compaction depends on everything under the sun, so stub everything with dummies.
59      StoreFile sf = mock(StoreFile.class);
60      StoreFile.Reader r = mock(StoreFile.Reader.class);
61      when(r.length()).thenReturn(1L);
62      when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
63      when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
64      when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong()))
65          .thenReturn(mock(StoreFileScanner.class));
66      when(sf.getReader()).thenReturn(r);
67      when(sf.createReader()).thenReturn(r);
68      when(sf.createReader(anyBoolean())).thenReturn(r);
69      when(sf.cloneForReader()).thenReturn(sf);
70      when(sf.getMaxSequenceId()).thenReturn(maxSequenceId);
71      return sf;
72    }
73  
74    public static CompactionRequest createDummyRequest() throws Exception {
75      return new CompactionRequest(Arrays.asList(createDummyStoreFile(1L)));
76    }
77  
78    // StoreFile.Writer has private ctor and is unwieldy, so this has to be convoluted.
79    public static class StoreFileWritersCapture
80        implements Answer<StoreFile.Writer>, AbstractMultiFileWriter.WriterFactory {
81      public static class Writer {
82        public ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
83        public TreeMap<byte[], byte[]> data = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
84        public boolean hasMetadata;
85      }
86  
87      private List<Writer> writers = new ArrayList<Writer>();
88  
89      @Override
90      public StoreFile.Writer createWriter() throws IOException {
91        final Writer realWriter = new Writer();
92        writers.add(realWriter);
93        StoreFile.Writer writer = mock(StoreFile.Writer.class);
94        doAnswer(new Answer<Object>() {
95          public Object answer(InvocationOnMock invocation) {
96            return realWriter.kvs.add((KeyValue) invocation.getArguments()[0]);
97          }
98        }).when(writer).append(any(KeyValue.class));
99        doAnswer(new Answer<Object>() {
100         public Object answer(InvocationOnMock invocation) {
101           Object[] args = invocation.getArguments();
102           return realWriter.data.put((byte[]) args[0], (byte[]) args[1]);
103         }
104       }).when(writer).appendFileInfo(any(byte[].class), any(byte[].class));
105       doAnswer(new Answer<Void>() {
106         @Override
107         public Void answer(InvocationOnMock invocation) throws Throwable {
108           realWriter.hasMetadata = true;
109           return null;
110         }
111       }).when(writer).appendMetadata(any(Long.class), any(boolean.class));
112       doAnswer(new Answer<Path>() {
113         @Override
114         public Path answer(InvocationOnMock invocation) throws Throwable {
115           return new Path("foo");
116         }
117       }).when(writer).getPath();
118       return writer;
119     }
120 
121     @Override
122     public StoreFile.Writer answer(InvocationOnMock invocation) throws Throwable {
123       return createWriter();
124     }
125 
126     public void verifyKvs(KeyValue[][] kvss, boolean allFiles, boolean requireMetadata) {
127       if (allFiles) {
128         assertEquals(kvss.length, writers.size());
129       }
130       int skippedWriters = 0;
131       for (int i = 0; i < kvss.length; ++i) {
132         KeyValue[] kvs = kvss[i];
133         if (kvs != null) {
134           Writer w = writers.get(i - skippedWriters);
135           if (requireMetadata) {
136             assertNotNull(w.data.get(STRIPE_START_KEY));
137             assertNotNull(w.data.get(STRIPE_END_KEY));
138           } else {
139             assertNull(w.data.get(STRIPE_START_KEY));
140             assertNull(w.data.get(STRIPE_END_KEY));
141           }
142           assertEquals(kvs.length, w.kvs.size());
143           for (int j = 0; j < kvs.length; ++j) {
144             assertEquals(kvs[j], w.kvs.get(j));
145           }
146         } else {
147           assertFalse(allFiles);
148           ++skippedWriters;
149         }
150       }
151     }
152 
153     public void verifyBoundaries(byte[][] boundaries) {
154       assertEquals(boundaries.length - 1, writers.size());
155       for (int i = 0; i < writers.size(); ++i) {
156         assertArrayEquals("i = " + i, boundaries[i], writers.get(i).data.get(STRIPE_START_KEY));
157         assertArrayEquals("i = " + i, boundaries[i + 1], writers.get(i).data.get(STRIPE_END_KEY));
158       }
159     }
160 
161     public void verifyKvs(KeyValue[][] kvss, boolean allFiles, List<Long> boundaries) {
162       if (allFiles) {
163         assertEquals(kvss.length, writers.size());
164       }
165       int skippedWriters = 0;
166       for (int i = 0; i < kvss.length; ++i) {
167         KeyValue[] kvs = kvss[i];
168         if (kvs != null) {
169           Writer w = writers.get(i - skippedWriters);
170           assertEquals(kvs.length, w.kvs.size());
171           for (int j = 0; j < kvs.length; ++j) {
172             assertTrue(kvs[j].getTimestamp() >= boundaries.get(i));
173             assertTrue(kvs[j].getTimestamp() < boundaries.get(i + 1));
174             assertEquals(kvs[j], w.kvs.get(j));
175           }
176         } else {
177           assertFalse(allFiles);
178           ++skippedWriters;
179         }
180       }
181     }
182 
183     public List<Writer> getWriters() {
184       return writers;
185     }
186   }
187 
188   public static class Scanner implements InternalScanner {
189     private final ArrayList<KeyValue> kvs;
190 
191     public Scanner(KeyValue... kvs) {
192       this.kvs = new ArrayList<KeyValue>(Arrays.asList(kvs));
193     }
194 
195     @Override
196     public boolean next(List<Cell> results) throws IOException {
197       if (kvs.isEmpty()) return false;
198       results.add(kvs.remove(0));
199       return !kvs.isEmpty();
200     }
201 
202     @Override
203     public boolean next(List<Cell> result, int limit) throws IOException {
204       return next(result);
205     }
206 
207     @Override
208     public void close() throws IOException {
209     }
210   }
211 }