11
11
import java .util .concurrent .TimeUnit ;
12
12
13
13
/**
14
- * Extended input stream.
14
+ * Extended input stream for read optimization .
15
15
*/
16
16
public abstract class ClickHouseInputStream extends InputStream {
17
+ /**
18
+ * Empty byte array.
19
+ */
20
+ public static final byte [] EMPTY_BYTES = new byte [0 ];
17
21
/**
18
22
* Empty and read-only byte buffer.
19
23
*/
20
- public static final ByteBuffer EMPTY = ByteBuffer .wrap (new byte [ 0 ] ).asReadOnlyBuffer ();
24
+ public static final ByteBuffer EMPTY_BUFFER = ByteBuffer .wrap (EMPTY_BYTES ).asReadOnlyBuffer ();
21
25
22
26
static final class BlockingInputStream extends ClickHouseInputStream {
23
27
private final BlockingQueue <ByteBuffer > queue ;
@@ -28,7 +32,7 @@ static final class BlockingInputStream extends ClickHouseInputStream {
28
32
private boolean closed ;
29
33
30
34
BlockingInputStream (BlockingQueue <ByteBuffer > queue , int timeout ) {
31
- this .queue = queue ;
35
+ this .queue = ClickHouseChecker . nonNull ( queue , "Queue" ) ;
32
36
this .timeout = timeout ;
33
37
34
38
this .buffer = null ;
@@ -37,10 +41,12 @@ static final class BlockingInputStream extends ClickHouseInputStream {
37
41
38
42
private void ensureOpen () throws IOException {
39
43
if (closed ) {
40
- throw new IOException ("Stream has been closed" );
44
+ throw new IOException (
45
+ ClickHouseUtils .format ("Blocking stream(queue: %d, buffer: %d) has been closed" ,
46
+ queue .size (), buffer != null ? buffer .remaining () : 0 ));
41
47
}
42
48
43
- if (buffer == null || (buffer != EMPTY && !buffer .hasRemaining ())) {
49
+ if (buffer == null || (buffer != EMPTY_BUFFER && !buffer .hasRemaining ())) {
44
50
updateBuffer ();
45
51
}
46
52
}
@@ -65,9 +71,11 @@ private int updateBuffer() throws IOException {
65
71
66
72
@ Override
67
73
public int available () throws IOException {
68
- ensureOpen ();
74
+ if (closed || buffer == EMPTY_BUFFER ) {
75
+ return 0 ;
76
+ }
69
77
70
- return buffer .remaining ();
78
+ return ( buffer == null || ! buffer . hasRemaining ()) ? updateBuffer () : buffer .remaining ();
71
79
}
72
80
73
81
@ Override
@@ -87,7 +95,7 @@ public void close() throws IOException {
87
95
public byte readByte () throws IOException {
88
96
ensureOpen ();
89
97
90
- if (buffer == EMPTY ) {
98
+ if (buffer == EMPTY_BUFFER ) {
91
99
close ();
92
100
throw new EOFException ();
93
101
}
@@ -99,7 +107,7 @@ public byte readByte() throws IOException {
99
107
public int read () throws IOException {
100
108
ensureOpen ();
101
109
102
- if (buffer == EMPTY ) {
110
+ if (buffer == EMPTY_BUFFER ) {
103
111
return -1 ;
104
112
}
105
113
@@ -112,7 +120,7 @@ public int read(byte[] b, int off, int len) throws IOException {
112
120
113
121
int counter = 0 ;
114
122
while (len > 0 ) {
115
- if (buffer == EMPTY ) {
123
+ if (buffer == EMPTY_BUFFER ) {
116
124
return counter > 0 ? counter : -1 ;
117
125
}
118
126
@@ -162,7 +170,7 @@ public long skip(long n) throws IOException {
162
170
// peforms better but this is a bit tricky
163
171
if (n == Long .MAX_VALUE ) {
164
172
long counter = buffer .remaining ();
165
- while (buffer != EMPTY && buffer .limit () > 0 ) {
173
+ while (buffer != EMPTY_BUFFER && buffer .limit () > 0 ) {
166
174
counter += buffer .limit ();
167
175
updateBuffer ();
168
176
}
@@ -184,20 +192,32 @@ static final class WrappedInputStream extends ClickHouseInputStream {
184
192
closed = false ;
185
193
}
186
194
195
+ private void ensureOpen () throws IOException {
196
+ if (closed ) {
197
+ throw new IOException (ClickHouseUtils .format ("Wrapped stream(%s) has been closed" , in ));
198
+ }
199
+ }
200
+
187
201
@ Override
188
202
public int available () throws IOException {
189
203
return !closed ? in .available () : 0 ;
190
204
}
191
205
192
206
@ Override
193
207
public byte readByte () throws IOException {
208
+ ensureOpen ();
209
+
194
210
int v = in .read ();
195
- if (v == -1 ) {
196
- close ();
197
- throw new EOFException ();
211
+ if (v != -1 ) {
212
+ return (byte ) v ;
198
213
}
199
214
200
- return (byte ) v ;
215
+ try {
216
+ close ();
217
+ } catch (IOException e ) {
218
+ // ignore
219
+ }
220
+ throw new EOFException ();
201
221
}
202
222
203
223
@ Override
@@ -207,25 +227,30 @@ public boolean isClosed() {
207
227
208
228
@ Override
209
229
public void close () throws IOException {
210
- try {
211
- in .close ();
212
- } finally {
213
- closed = true ;
230
+ if (!closed ) {
231
+ try {
232
+ in .close ();
233
+ } finally {
234
+ closed = true ;
235
+ }
214
236
}
215
237
}
216
238
217
239
@ Override
218
240
public int read () throws IOException {
241
+ ensureOpen ();
219
242
return in .read ();
220
243
}
221
244
222
245
@ Override
223
246
public int read (byte [] b , int off , int len ) throws IOException {
247
+ ensureOpen ();
224
248
return in .read (b , off , len );
225
249
}
226
250
227
251
@ Override
228
252
public long skip (long n ) throws IOException {
253
+ ensureOpen ();
229
254
return in .skip (n );
230
255
}
231
256
}
@@ -238,21 +263,23 @@ public long skip(long n) throws IOException {
238
263
* @return wrapped input
239
264
*/
240
265
public static ClickHouseInputStream of (BlockingQueue <ByteBuffer > queue , int timeout ) {
241
- return new BlockingInputStream (ClickHouseChecker . nonNull ( queue , "queue" ) , timeout );
266
+ return new BlockingInputStream (queue , timeout );
242
267
}
243
268
244
269
/**
245
270
* Wraps the given input stream.
246
271
*
247
272
* @param input non-null input stream
248
- * @return wrapped input
273
+ * @return wrapped input, or the same input if it's instance of
274
+ * {@link ClickHouseInputStream}
249
275
*/
250
276
public static ClickHouseInputStream of (InputStream input ) {
251
277
return input instanceof ClickHouseInputStream ? (ClickHouseInputStream ) input : new WrappedInputStream (input );
252
278
}
253
279
254
280
/**
255
- * Reads an unsigned byte from the input stream.
281
+ * Reads an unsigned byte from the input stream. Unlike {@link #read()}, it will
282
+ * throw {@link IOException} if the input stream has been closed.
256
283
*
257
284
* @return unsigned byte
258
285
* @throws IOException when failed to read value from input stream or reached
@@ -263,8 +290,10 @@ public int readUnsignedByte() throws IOException {
263
290
}
264
291
265
292
/**
266
- * Reads one single byte from the input stream. It's supposed to be faster than
267
- * {@link #read()}.
293
+ * Reads one single byte from the input stream. Unlike {@link #read()}, it will
294
+ * throw {@link IOException} if the input stream has been closed. In general,
295
+ * this method should be faster than {@link #read()}, especially when it's an
296
+ * input stream backed by byte[] or {@link java.nio.ByteBuffer}.
268
297
*
269
298
* @return byte value if present
270
299
* @throws IOException when failed to read value from input stream or reached
@@ -274,14 +303,19 @@ public int readUnsignedByte() throws IOException {
274
303
275
304
/**
276
305
* Reads {@code length} bytes from the input stream. It behaves in the same
277
- * way as {@link java.io.DataInput#readFully(byte[])}.
306
+ * way as {@link java.io.DataInput#readFully(byte[])}, and it will throw
307
+ * {@link IOException} when the input stream has been closed.
278
308
*
279
309
* @param length number of bytes to read
280
310
* @return byte array and its length should be {@code length}
281
311
* @throws IOException when failed to read value from input stream, not able to
282
312
* retrieve all bytes, or reached end of the stream
283
313
*/
284
314
public byte [] readBytes (int length ) throws IOException {
315
+ if (length <= 0 ) {
316
+ return EMPTY_BYTES ;
317
+ }
318
+
285
319
byte [] bytes = new byte [length ];
286
320
287
321
for (int l = length , c = 0 , n = 0 ; l > 0 ; l -= n ) {
0 commit comments