1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.io;
19
20 import java.io.IOException;
21 import java.lang.reflect.Method;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.fs.FSDataInputStream;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.fs.HFileSystem;
29
30 import com.google.common.annotations.VisibleForTesting;
31
32
33
34
35
36
37 public class FSDataInputStreamWrapper {
38 static final Log LOG = LogFactory.getLog(FSDataInputStreamWrapper.class);
39
40 private final HFileSystem hfs;
41 private final Path path;
42 private final FileLink link;
43 private final boolean doCloseStreams;
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65 private volatile FSDataInputStream stream = null;
66 private volatile FSDataInputStream streamNoFsChecksum = null;
67 private Object streamNoFsChecksumFirstCreateLock = new Object();
68
69
70 private boolean useHBaseChecksumConfigured;
71
72
73
74
75
76 private volatile boolean useHBaseChecksum;
77
78
79
80 private volatile int hbaseChecksumOffCount = -1;
81
82 public FSDataInputStreamWrapper(FileSystem fs, Path path) throws IOException {
83 this(fs, null, path, false);
84 }
85
86 public FSDataInputStreamWrapper(FileSystem fs, Path path, boolean dropBehind) throws IOException {
87 this(fs, null, path, dropBehind);
88 }
89
90 public FSDataInputStreamWrapper(FileSystem fs, FileLink link) throws IOException {
91 this(fs, link, null, false);
92 }
93 public FSDataInputStreamWrapper(FileSystem fs, FileLink link,
94 boolean dropBehind) throws IOException {
95 this(fs, link, null, dropBehind);
96 }
97
98 private FSDataInputStreamWrapper(FileSystem fs, FileLink link,
99 Path path, boolean dropBehind) throws IOException {
100 assert (path == null) != (link == null);
101 this.path = path;
102 this.link = link;
103 this.doCloseStreams = true;
104
105
106
107 this.hfs = (fs instanceof HFileSystem) ? (HFileSystem)fs : new HFileSystem(fs);
108
109
110 this.useHBaseChecksumConfigured = this.useHBaseChecksum = false;
111 this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
112 try {
113 Class<? extends FSDataInputStream> inputStreamClass = this.stream.getClass();
114 try {
115 Method m = inputStreamClass.getDeclaredMethod("setDropBehind",
116 new Class[] { boolean.class });
117 m.invoke(stream, new Object[] { dropBehind });
118 } catch (NoSuchMethodException e) {
119
120 } catch (Exception e) {
121 if (LOG.isDebugEnabled()) {
122 LOG.debug("Failed to invoke input stream's setDropBehind method, continuing");
123 }
124 }
125 } catch (Exception e) {
126
127 }
128 }
129
130
131
132
133
134
135
136
137 public void prepareForBlockReader(boolean forceNoHBaseChecksum) throws IOException {
138 if (hfs == null) return;
139 assert this.stream != null && !this.useHBaseChecksumConfigured;
140 boolean useHBaseChecksum =
141 !forceNoHBaseChecksum && hfs.useHBaseChecksum() && (hfs.getNoChecksumFs() != hfs);
142
143 if (useHBaseChecksum) {
144 FileSystem fsNc = hfs.getNoChecksumFs();
145 this.streamNoFsChecksum = (link != null) ? link.open(fsNc) : fsNc.open(path);
146 this.useHBaseChecksumConfigured = this.useHBaseChecksum = useHBaseChecksum;
147
148 this.stream.close();
149 this.stream = null;
150 }
151 }
152
153
154 @VisibleForTesting
155 public FSDataInputStreamWrapper(FSDataInputStream fsdis) {
156 this(fsdis, fsdis);
157 }
158
159
160 @VisibleForTesting
161 public FSDataInputStreamWrapper(FSDataInputStream fsdis, FSDataInputStream noChecksum) {
162 doCloseStreams = false;
163 stream = fsdis;
164 streamNoFsChecksum = noChecksum;
165 path = null;
166 link = null;
167 hfs = null;
168 useHBaseChecksumConfigured = useHBaseChecksum = false;
169 }
170
171
172
173
174 public boolean shouldUseHBaseChecksum() {
175 return this.useHBaseChecksum;
176 }
177
178
179
180
181
182
183 public FSDataInputStream getStream(boolean useHBaseChecksum) {
184 return useHBaseChecksum ? this.streamNoFsChecksum : this.stream;
185 }
186
187
188
189
190
191 public FSDataInputStream fallbackToFsChecksum(int offCount) throws IOException {
192
193 boolean partOfConvoy = false;
194 if (this.stream == null) {
195 synchronized (streamNoFsChecksumFirstCreateLock) {
196 partOfConvoy = (this.stream != null);
197 if (!partOfConvoy) {
198 this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
199 }
200 }
201 }
202 if (!partOfConvoy) {
203 this.useHBaseChecksum = false;
204 this.hbaseChecksumOffCount = offCount;
205 }
206 return this.stream;
207 }
208
209
210 public void checksumOk() {
211 if (this.useHBaseChecksumConfigured && !this.useHBaseChecksum
212 && (this.hbaseChecksumOffCount-- < 0)) {
213
214 assert this.streamNoFsChecksum != null;
215 this.useHBaseChecksum = true;
216 }
217 }
218
219
220 public void close() throws IOException {
221 if (!doCloseStreams) return;
222 try {
223 if (stream != streamNoFsChecksum && streamNoFsChecksum != null) {
224 streamNoFsChecksum.close();
225 streamNoFsChecksum = null;
226 }
227 } finally {
228 if (stream != null) {
229 stream.close();
230 stream = null;
231 }
232 }
233 }
234
235 public HFileSystem getHfs() {
236 return this.hfs;
237 }
238 }