summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorJenkins <jenkins@review.openstack.org>2016-11-28 15:32:29 +0000
committerGerrit Code Review <review@openstack.org>2016-11-28 15:32:29 +0000
commitd95ffe7014c805f92d5e8a91fdd9bae8fb7e6384 (patch)
treee867a353c14a1a77bc5d3510224a6a35505b7e77
parentac8b0480df83673a9d0018b0af13dcdad8f94d41 (diff)
parentef3c1ab4d53a59b7aa47186d184fe950ded8ad91 (diff)
Merge "Adds the lazy seek"6.0.0.0b2
-rw-r--r--hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeInputStream.java22
1 files changed, 22 insertions, 0 deletions
diff --git a/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeInputStream.java b/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeInputStream.java
index 691ecd0..04fbfc9 100644
--- a/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeInputStream.java
+++ b/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeInputStream.java
@@ -89,6 +89,8 @@ class SwiftNativeInputStream extends FSInputStream {
89 */ 89 */
90 private long rangeOffset = 0; 90 private long rangeOffset = 0;
91 91
92 private long nextReadPosition = 0;
93
92 public SwiftNativeInputStream(SwiftNativeFileSystemStore storeNative, 94 public SwiftNativeInputStream(SwiftNativeFileSystemStore storeNative,
93 FileSystem.Statistics statistics, Path path, long bufferSize) 95 FileSystem.Statistics statistics, Path path, long bufferSize)
94 throws IOException { 96 throws IOException {
@@ -138,6 +140,7 @@ class SwiftNativeInputStream extends FSInputStream {
138 verifyOpen(); 140 verifyOpen();
139 int result = -1; 141 int result = -1;
140 try { 142 try {
143 seekStream();
141 result = httpStream.read(); 144 result = httpStream.read();
142 } catch (IOException e) { 145 } catch (IOException e) {
143 String msg = "IOException while reading " + path 146 String msg = "IOException while reading " + path
@@ -300,6 +303,13 @@ class SwiftNativeInputStream extends FSInputStream {
300 if (targetPos < 0) { 303 if (targetPos < 0) {
301 throw new IOException("Negative Seek offset not supported"); 304 throw new IOException("Negative Seek offset not supported");
302 } 305 }
306 nextReadPosition = targetPos;
307 }
308
309 public synchronized void realSeek(long targetPos) throws IOException {
310 if (targetPos < 0) {
311 throw new IOException("Negative Seek offset not supported");
312 }
303 //there's some special handling of near-local data 313 //there's some special handling of near-local data
304 //as the seek can be omitted if it is in/adjacent 314 //as the seek can be omitted if it is in/adjacent
305 long offset = targetPos - pos; 315 long offset = targetPos - pos;
@@ -345,6 +355,18 @@ class SwiftNativeInputStream extends FSInputStream {
345 } 355 }
346 356
347 /** 357 /**
358 * Lazy seek.
359 * @throws IOException
360 */
361 private void seekStream() throws IOException {
362 if (httpStream != null && nextReadPosition == pos) {
363 // already at specified position
364 return;
365 }
366 realSeek(nextReadPosition);
367 }
368
369 /**
348 * Fill the buffer from the target position 370 * Fill the buffer from the target position
349 * If the target position == current position, the 371 * If the target position == current position, the
350 * read still goes ahead; this is a way of handling partial read failures 372 * read still goes ahead; this is a way of handling partial read failures