View Javadoc

1   /**
2    * Copyright 2008 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.io;
21  
22  import java.io.IOException;
23  import java.nio.ByteBuffer;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.fs.FileSystem;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.KeyValue;
30  import org.apache.hadoop.hbase.io.hfile.BlockCache;
31  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
32  import org.apache.hadoop.hbase.regionserver.StoreFile;
33  import org.apache.hadoop.hbase.util.Bytes;
34  
35  /**
36   * A facade for a {@link org.apache.hadoop.hbase.io.hfile.HFile.Reader} that serves up
37   * either the top or bottom half of a HFile where 'bottom' is the first half
38   * of the file containing the keys that sort lowest and 'top' is the second half
39   * of the file with keys that sort greater than those of the bottom half.
40   * The top includes the split files midkey, of the key that follows if it does
41   * not exist in the file.
42   *
43   * <p>This type works in tandem with the {@link Reference} type.  This class
44   * is used reading while Reference is used writing.
45   *
46   * <p>This file is not splitable.  Calls to {@link #midkey()} return null.
47   */
48  public class HalfStoreFileReader extends StoreFile.Reader {
49    final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
50    final boolean top;
51    // This is the key we split around.  Its the first possible entry on a row:
52    // i.e. empty column and a timestamp of LATEST_TIMESTAMP.
53    protected final byte [] splitkey;
54  
55    /**
56     * @param fs
57     * @param p
58     * @param c
59     * @param r
60     * @throws IOException
61     */
62    public HalfStoreFileReader(final FileSystem fs, final Path p, final BlockCache c,
63      final Reference r)
64    throws IOException {
65      super(fs, p, c, false);
66      // This is not actual midkey for this half-file; its just border
67      // around which we split top and bottom.  Have to look in files to find
68      // actual last and first keys for bottom and top halves.  Half-files don't
69      // have an actual midkey themselves. No midkey is how we indicate file is
70      // not splittable.
71      this.splitkey = r.getSplitKey();
72      // Is it top or bottom half?
73      this.top = Reference.isTopFileRegion(r.getFileRegion());
74    }
75  
76    protected boolean isTop() {
77      return this.top;
78    }
79  
80    @Override
81    public HFileScanner getScanner(final boolean cacheBlocks, final boolean pread) {
82      final HFileScanner s = super.getScanner(cacheBlocks, pread);
83      return new HFileScanner() {
84        final HFileScanner delegate = s;
85        public boolean atEnd = false;
86  
87        public ByteBuffer getKey() {
88          if (atEnd) return null;
89          return delegate.getKey();
90        }
91  
92        public String getKeyString() {
93          if (atEnd) return null;
94  
95          return delegate.getKeyString();
96        }
97  
98        public ByteBuffer getValue() {
99          if (atEnd) return null;
100 
101         return delegate.getValue();
102       }
103 
104       public String getValueString() {
105         if (atEnd) return null;
106 
107         return delegate.getValueString();
108       }
109 
110       public KeyValue getKeyValue() {
111         if (atEnd) return null;
112 
113         return delegate.getKeyValue();
114       }
115 
116       public boolean next() throws IOException {
117         if (atEnd) return false;
118 
119         boolean b = delegate.next();
120         if (!b) {
121           return b;
122         }
123         // constrain the bottom.
124         if (!top) {
125           ByteBuffer bb = getKey();
126           if (getComparator().compare(bb.array(), bb.arrayOffset(), bb.limit(),
127               splitkey, 0, splitkey.length) >= 0) {
128             atEnd = true;
129             return false;
130           }
131         }
132         return true;
133       }
134 
135       public boolean seekBefore(byte[] key) throws IOException {
136         return seekBefore(key, 0, key.length);
137       }
138 
139       public boolean seekBefore(byte [] key, int offset, int length)
140       throws IOException {
141         if (top) {
142           if (getComparator().compare(key, offset, length, splitkey, 0,
143               splitkey.length) < 0) {
144             return false;
145           }
146         } else {
147           if (getComparator().compare(key, offset, length, splitkey, 0,
148               splitkey.length) >= 0) {
149             return seekBefore(splitkey, 0, splitkey.length);
150           }
151         }
152         return this.delegate.seekBefore(key, offset, length);
153       }
154 
155       public boolean seekTo() throws IOException {
156         if (top) {
157           int r = this.delegate.seekTo(splitkey);
158           if (r < 0) {
159             // midkey is < first key in file
160             return this.delegate.seekTo();
161           }
162           if (r > 0) {
163             return this.delegate.next();
164           }
165           return true;
166         }
167 
168         boolean b = delegate.seekTo();
169         if (!b) {
170           return b;
171         }
172         // Check key.
173         ByteBuffer k = this.delegate.getKey();
174         return this.delegate.getReader().getComparator().
175           compare(k.array(), k.arrayOffset(), k.limit(),
176             splitkey, 0, splitkey.length) < 0;
177       }
178 
179       public int seekTo(byte[] key) throws IOException {
180         return seekTo(key, 0, key.length);
181       }
182 
183       public int seekTo(byte[] key, int offset, int length) throws IOException {
184         if (top) {
185           if (getComparator().compare(key, offset, length, splitkey, 0,
186               splitkey.length) < 0) {
187             return -1;
188           }
189         } else {
190           if (getComparator().compare(key, offset, length, splitkey, 0,
191               splitkey.length) >= 0) {
192             // we would place the scanner in the second half.
193             // it might be an error to return false here ever...
194             boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
195             if (!res) {
196               throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
197             }
198             return 1;
199           }
200         }
201         return delegate.seekTo(key, offset, length);
202       }
203 
204       @Override
205       public int reseekTo(byte[] key) throws IOException {
206         return reseekTo(key, 0, key.length);
207       }
208 
209       @Override
210       public int reseekTo(byte[] key, int offset, int length)
211       throws IOException {
212         //This function is identical to the corresponding seekTo function except
213         //that we call reseekTo (and not seekTo) on the delegate.
214         if (top) {
215           if (getComparator().compare(key, offset, length, splitkey, 0,
216               splitkey.length) < 0) {
217             return -1;
218           }
219         } else {
220           if (getComparator().compare(key, offset, length, splitkey, 0,
221               splitkey.length) >= 0) {
222             // we would place the scanner in the second half.
223             // it might be an error to return false here ever...
224             boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
225             if (!res) {
226               throw new IOException("Seeking for a key in bottom of file, but" +
227                   " key exists in top of file, failed on seekBefore(midkey)");
228             }
229             return 1;
230           }
231         }
232         if (atEnd) {
233           // skip the 'reseek' and just return 1.
234           return 1;
235         }
236         return delegate.reseekTo(key, offset, length);
237       }
238 
239       public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
240         return this.delegate.getReader();
241       }
242 
243       public boolean isSeeked() {
244         return this.delegate.isSeeked();
245       }
246     };
247   }
248 
249   @Override
250   public byte[] getLastKey() {
251     if (top) {
252       return super.getLastKey();
253     }
254     // Get a scanner that caches the block and that uses pread.
255     HFileScanner scanner = getScanner(true, true);
256     try {
257       if (scanner.seekBefore(this.splitkey)) {
258         return Bytes.toBytes(scanner.getKey());
259       }
260     } catch (IOException e) {
261       LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
262     }
263     return null;
264   }
265 
266   @Override
267   public byte[] midkey() throws IOException {
268     // Returns null to indicate file is not splitable.
269     return null;
270   }
271 }