-
Notifications
You must be signed in to change notification settings - Fork 26
CASSANALYTICS-147: BufferingInputStream fails to read last chunk #193
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: trunk
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,7 +20,10 @@ | |
| package org.apache.cassandra.spark.utils; | ||
|
|
||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.time.Duration; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.ScheduledExecutorService; | ||
|
|
@@ -33,6 +36,7 @@ | |
| import com.google.common.collect.ImmutableList; | ||
| import com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
| import com.google.common.util.concurrent.Uninterruptibles; | ||
| import org.apache.commons.lang.ArrayUtils; | ||
| import org.apache.commons.lang.mutable.MutableInt; | ||
| import org.junit.jupiter.api.Test; | ||
|
|
||
|
|
@@ -289,7 +293,7 @@ public Duration timeout() | |
| }; | ||
|
|
||
| int bytesToRead = chunkSize * numChunks; | ||
| long skipAhead = size - bytesToRead + 1; | ||
| long skipAhead = size - bytesToRead; | ||
| try (BufferingInputStream<SSTable> stream = new BufferingInputStream<>(source, STATS.bufferingInputStreamStats())) | ||
| { | ||
| // Skip ahead so we only read the final chunks | ||
|
|
@@ -346,6 +350,77 @@ public Duration timeout() | |
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testUnalignedEndReading() throws IOException | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Minor: We might want to assert on |
||
| { | ||
| int dataSize = 2527234; | ||
| int chunkSize = 4096; | ||
| int remainingReadBytes = 2; | ||
| List<byte[]> returnedBuffers = new ArrayList<>(); | ||
| CassandraFileSource<SSTable> source = new CassandraFileSource<SSTable>() | ||
| { | ||
| @Override | ||
| public void request(long start, long end, StreamConsumer consumer) | ||
| { | ||
| byte[] bytes = RandomUtils.randomBytes((int) (end - start + 1)); | ||
| StreamBuffer buffer = StreamBuffer.wrap(bytes); | ||
| returnedBuffers.add(bytes); | ||
| consumer.onRead(buffer); | ||
| consumer.onEnd(); | ||
| } | ||
|
|
||
| @Override | ||
| public SSTable cassandraFile() | ||
| { | ||
| return null; | ||
| } | ||
|
|
||
| @Override | ||
| public FileType fileType() | ||
| { | ||
| return FileType.PARTITIONS_INDEX; | ||
| } | ||
|
|
||
| @Override | ||
| public long size() | ||
| { | ||
| return dataSize; | ||
| } | ||
|
|
||
| @Override | ||
| @Nullable | ||
| public Duration timeout() | ||
| { | ||
| return Duration.ofSeconds(5); | ||
| } | ||
|
|
||
| @Override | ||
| public long chunkBufferSize() | ||
| { | ||
| return chunkSize; | ||
| } | ||
| }; | ||
|
|
||
| try (BufferingInputStream<SSTable> stream1 = new BufferingInputStream<>(source, STATS.bufferingInputStreamStats())) | ||
| { | ||
| // move left from the file end by (chunkSize + remainingReadBytes) | ||
| try (BufferingInputStream<SSTable> stream2 = stream1.reBuffer(dataSize - chunkSize - remainingReadBytes)) | ||
| { | ||
| ByteBuffer buffer = ByteBuffer.allocate(chunkSize); | ||
| int read = stream2.read(buffer); // read last full chunk | ||
| assertThat(returnedBuffers).hasSize(2); | ||
| assertThat(read).isEqualTo(chunkSize); | ||
| assertThat(buffer.array()).isEqualTo(returnedBuffers.get(0)); | ||
|
|
||
| buffer.position(0); | ||
| buffer.limit(remainingReadBytes); | ||
| read = stream2.read(buffer); // read remaining bytes | ||
| assertThat(read).isEqualTo(remainingReadBytes); | ||
| assertThat(ArrayUtils.subarray(buffer.array(), 0, remainingReadBytes)).isEqualTo(returnedBuffers.get(1)); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // Utils | ||
|
|
||
| private static ImmutableList<StreamBuffer> randomBuffers(int count) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -92,9 +92,9 @@ public void testCDCRebufferSequentialReading() throws IOException | |
| long position = call.start; | ||
|
|
||
| // Deliver data in chunks until request is fulfilled | ||
| while (position < actualEnd) | ||
| while (position <= actualEnd) // range boundaries are inclusive | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. According to below JavaDoc, ranges should be considered inclusive. |
||
| { | ||
| int chunkSize = (int) Math.min(actualEnd - position, bufferSize); | ||
| int chunkSize = (int) Math.min(actualEnd - position + 1, bufferSize); | ||
| Buffer data = Buffer.buffer(); | ||
| for (int i = 0; i < chunkSize; i++) | ||
| { | ||
|
|
@@ -157,7 +157,7 @@ public void testCDCRebufferBackwardSeek() throws IOException | |
| // Backward seek path has a bug: requests buffer.remaining() + 1 bytes | ||
| // We cap delivery at buffer capacity to work around this and test flip() behavior | ||
| long actualEnd = Math.min(call.end, testCommitLog.maxOffset()); | ||
| long requestedBytes = actualEnd - call.start; | ||
| long requestedBytes = actualEnd - call.start + 1; // range boundaries are inclusive | ||
| long position = call.start; | ||
|
|
||
| // Cap delivery to buffer capacity to avoid BufferOverflowException from + 1 bug | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am not sure how effective the change in
BufferingInputStreamwould affectskip()used during BIG index reading. All integration tests pass though, and I think hereby unit test is just a simulation.