1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.regionserver;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.hbase.KeyValue;
26 import org.apache.hadoop.hbase.client.Scan;
27 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
28
29 import java.io.IOException;
30 import java.util.ArrayList;
31 import java.util.Collection;
32 import java.util.List;
33 import java.util.SortedSet;
34
35
36
37
38
39 class StoreFileScanner implements KeyValueScanner {
40 static final Log LOG = LogFactory.getLog(Store.class);
41
42
43 private final StoreFile.Reader reader;
44 private final HFileScanner hfs;
45 private KeyValue cur = null;
46
47
48
49
50
51 public StoreFileScanner(StoreFile.Reader reader, HFileScanner hfs) {
52 this.reader = reader;
53 this.hfs = hfs;
54 }
55
56
57
58
59
60 public static List<StoreFileScanner> getScannersForStoreFiles(
61 Collection<StoreFile> filesToCompact,
62 boolean cacheBlocks,
63 boolean usePread) throws IOException {
64 List<StoreFileScanner> scanners =
65 new ArrayList<StoreFileScanner>(filesToCompact.size());
66 for (StoreFile file : filesToCompact) {
67 StoreFile.Reader r = file.createReader();
68 scanners.add(r.getStoreFileScanner(cacheBlocks, usePread));
69 }
70 return scanners;
71 }
72
73 public String toString() {
74 return "StoreFileScanner[" + hfs.toString() + ", cur=" + cur + "]";
75 }
76
77 public KeyValue peek() {
78 return cur;
79 }
80
81 public KeyValue next() throws IOException {
82 KeyValue retKey = cur;
83 cur = hfs.getKeyValue();
84 try {
85
86 if (cur != null)
87 hfs.next();
88 } catch(IOException e) {
89 throw new IOException("Could not iterate " + this, e);
90 }
91 return retKey;
92 }
93
94 public boolean seek(KeyValue key) throws IOException {
95 try {
96 if(!seekAtOrAfter(hfs, key)) {
97 close();
98 return false;
99 }
100 cur = hfs.getKeyValue();
101 hfs.next();
102 return true;
103 } catch(IOException ioe) {
104 throw new IOException("Could not seek " + this, ioe);
105 }
106 }
107
108 public boolean reseek(KeyValue key) throws IOException {
109 try {
110 if (!reseekAtOrAfter(hfs, key)) {
111 close();
112 return false;
113 }
114 cur = hfs.getKeyValue();
115 hfs.next();
116 return true;
117 } catch (IOException ioe) {
118 throw new IOException("Could not seek " + this, ioe);
119 }
120 }
121
122 public void close() {
123
124 cur = null;
125 }
126
127
128
129
130
131
132
133
134 public static boolean seekAtOrAfter(HFileScanner s, KeyValue k)
135 throws IOException {
136 int result = s.seekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
137 if(result < 0) {
138
139 return s.seekTo();
140 } else if(result > 0) {
141
142
143 return s.next();
144 }
145
146 return true;
147 }
148
149 static boolean reseekAtOrAfter(HFileScanner s, KeyValue k)
150 throws IOException {
151
152 int result = s.reseekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
153 if (result <= 0) {
154 return true;
155 } else {
156
157
158 return s.next();
159 }
160 }
161
162
163 public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
164 return reader.shouldSeek(scan, columns);
165 }
166
167 @Override
168 public long getSequenceID() {
169 return reader.getSequenceID();
170 }
171 }