this.input = new ChunkedSliceInput(new DataSourceSliceLoader(dataSource), toIntExact(bufferSize.toBytes())); Slice magic = input.readSlice(RCFILE_MAGIC.length()); boolean compressed; if (RCFILE_MAGIC.equals(magic)) { version = input.readByte(); verify(version <= CURRENT_VERSION, "RCFile version %s not supported: %s", version, dataSource); validateWrite(validation -> validation.getVersion() == version, "Unexpected file version"); compressed = input.readBoolean(); byte sequenceFileVersion = input.readByte(); verify(sequenceFileVersion == SEQUENCE_FILE_VERSION, "File %s is a SequenceFile not an RCFile", dataSource); Slice valueClassName = readLengthPrefixedString(input); verify(RCFILE_KEY_BUFFER_NAME.equals(keyClassName) && RCFILE_VALUE_BUFFER_NAME.equals(valueClassName), "File %s is a SequenceFile not an RCFile", dataSource); compressed = input.readBoolean(); if (input.readBoolean()) { throw corrupt("File %s is a SequenceFile not an RCFile", dataSource); int metadataEntries = Integer.reverseBytes(input.readInt()); verify(metadataEntries >= 0, "Invalid metadata entry count %s in RCFile %s", metadataEntries, dataSource); verify(metadataEntries <= MAX_METADATA_ENTRIES, "Too many metadata entries (%s) in RCFile %s", metadataEntries, dataSource); syncFirst = input.readLong(); validateWrite(validation -> validation.getSyncFirst() == syncFirst, "Unexpected sync sequence"); syncSecond = input.readLong(); validateWrite(validation -> validation.getSyncSecond() == syncSecond, "Unexpected sync sequence");
if (input.remaining() == 0) { close(); return -1; verify(input.remaining() >= SIZE_OF_INT, "RCFile truncated %s", dataSource.getId()); int unusedRowGroupSize = Integer.reverseBytes(input.readInt()); verify(input.remaining() >= SIZE_OF_LONG + SIZE_OF_LONG + SIZE_OF_INT, "RCFile truncated %s", dataSource.getId()); if (input.position() - SIZE_OF_INT >= end) { close(); return -1; verify(syncFirst == input.readLong() && syncSecond == input.readLong(), "Invalid sync in RCFile %s", dataSource.getId()); unusedRowGroupSize = Integer.reverseBytes(input.readInt()); int uncompressedHeaderSize = Integer.reverseBytes(input.readInt()); int compressedHeaderSize = Integer.reverseBytes(input.readInt()); if (compressedHeaderSize > compressedHeaderBuffer.length()) { compressedHeaderBuffer = Slices.allocate(compressedHeaderSize); input.readBytes(compressedHeaderBuffer, 0, compressedHeaderSize); Slice dataBuffer = input.readSlice(compressedDataSize); columns[columnIndex].setBuffers(lengthsBuffer, dataBuffer, uncompressedDataSize);
private void seekToFirstRowGroupInRange(long offset, long length) throws IOException { long startOfSyncSequence = findFirstSyncPosition(dataSource, offset, length, syncFirst, syncSecond); if (startOfSyncSequence < 0) { closeQuietly(); return; } input.setPosition(startOfSyncSequence); }
@Test public void testRetainedSize() { int length = 1024; int bufferSize = 128; Slice slice = Slices.utf8Slice(Strings.repeat("0", length)); SliceSliceLoader loader = new SliceSliceLoader(slice); ChunkedSliceInput chunkedSliceInput = new ChunkedSliceInput(loader, bufferSize); for (int i = 0; i < length; i++) { chunkedSliceInput.setPosition(i); chunkedSliceInput.readByte(); assertRetainedSize(chunkedSliceInput); long retainedSize = chunkedSliceInput.getRetainedSize(); assertTrue(retainedSize > bufferSize && retainedSize < length); } }
@Test public void testSetPosition() { // Create a ChunkedSliceInput with 160 bytes, and set bufferSize to 130. // This test read bytes sequentially, but calls setPosition every time. // Only 2 reads should be issued to the underlying SliceLoader. One when // trying to read position 0, and one when trying to read position 130. int length = 160; int bufferSize = 130; Slice slice = Slices.utf8Slice(Strings.repeat("0", length)); SliceSliceLoader loader = new SliceSliceLoader(slice); ChunkedSliceInput chunkedSliceInput = new ChunkedSliceInput(loader, bufferSize); ArrayList<Integer> actual = new ArrayList<>(); for (int i = 0; i < length; i++) { chunkedSliceInput.setPosition(i); chunkedSliceInput.readByte(); int count = loader.getCount(); actual.add(count); } List<Integer> expected = IntStream.range(0, length) .map(i -> i < bufferSize ? 1 : 2) .boxed() .collect(Collectors.toList()); assertEquals(actual, expected); }
@Override public FixedLengthSliceInput get() { return new ChunkedSliceInput(new ChunkedSliceLoader(diskRange), bufferSize); } }
@Override public void readBytes(OutputStream out, int length) throws IOException { checkBound(position() + length, globalLength, "End of stream"); while (length > 0) { int bytesToRead = min(available(), length); buffer.getBytes(bufferPosition, out, bytesToRead); bufferPosition += bytesToRead; length -= bytesToRead; ensureAvailable(min(length, buffer.length())); } }
@Override public int read(byte[] destination, int destinationIndex, int length) { if (length == 0) { return 0; } if (globalLength - position() == 0) { return -1; } // limit read to stream size length = (int) min(length, globalLength - position()); // do a full read of the available data readBytes(destination, destinationIndex, length); return length; }
@Override public boolean readBoolean() { return readByte() != 0; }
public void ensureAvailable(int size) { if (available() >= size) { return; } checkArgument(size <= buffer.length(), "Size is larger than buffer"); checkBound(position() + size, globalLength, "End of stream"); // advance position globalPosition += bufferPosition; bufferPosition = 0; // this will reread unused data in the buffer long readSize = min(buffer.length(), globalLength - globalPosition); if (readSize > Integer.MAX_VALUE) { readSize = Integer.MAX_VALUE; } bufferLength = (int) readSize; loader.load(globalPosition, bufferLength); }
@Override public int read() { if (position() >= globalLength) { return -1; } ensureAvailable(SIZE_OF_BYTE); int result = buffer.getByte(bufferPosition) & 0xFF; bufferPosition++; return result; }
@Override public long skip(long length) { // is skip within the current buffer? if (available() >= length) { bufferPosition += length; return length; } // drop current buffer globalPosition += bufferPosition; bufferPosition = 0; bufferLength = 0; // trim length to stream size length = min(length, remaining()); // skip globalPosition += length; return length; }
@Override public Slice readSlice(int length) { if (length == 0) { return Slices.EMPTY_SLICE; } Slice slice = Slices.allocate(length); readBytes(slice); return slice; }
currentChunkRowCount = 0; try { input.close();
@Test public void testRetainedSize() { int length = 1024; int bufferSize = 128; Slice slice = Slices.utf8Slice(Strings.repeat("0", length)); SliceSliceLoader loader = new SliceSliceLoader(slice); ChunkedSliceInput chunkedSliceInput = new ChunkedSliceInput(loader, bufferSize); for (int i = 0; i < length; i++) { chunkedSliceInput.setPosition(i); chunkedSliceInput.readByte(); assertRetainedSize(chunkedSliceInput); long retainedSize = chunkedSliceInput.getRetainedSize(); assertTrue(retainedSize > bufferSize && retainedSize < length); } }
@Test public void testSetPosition() { // Create a ChunkedSliceInput with 160 bytes, and set bufferSize to 130. // This test read bytes sequentially, but calls setPosition every time. // Only 2 reads should be issued to the underlying SliceLoader. One when // trying to read position 0, and one when trying to read position 130. int length = 160; int bufferSize = 130; Slice slice = Slices.utf8Slice(Strings.repeat("0", length)); SliceSliceLoader loader = new SliceSliceLoader(slice); ChunkedSliceInput chunkedSliceInput = new ChunkedSliceInput(loader, bufferSize); ArrayList<Integer> actual = new ArrayList<>(); for (int i = 0; i < length; i++) { chunkedSliceInput.setPosition(i); chunkedSliceInput.readByte(); int count = loader.getCount(); actual.add(count); } List<Integer> expected = IntStream.range(0, length) .map(i -> i < bufferSize ? 1 : 2) .boxed() .collect(Collectors.toList()); assertEquals(actual, expected); }
private <K> Map<K, FixedLengthSliceInput> readLargeDiskRanges(Map<K, DiskRange> diskRanges) throws IOException { if (diskRanges.isEmpty()) { return ImmutableMap.of(); } ImmutableMap.Builder<K, FixedLengthSliceInput> slices = ImmutableMap.builder(); for (Entry<K, DiskRange> entry : diskRanges.entrySet()) { ChunkedSliceInput sliceInput = new ChunkedSliceInput(new HdfsSliceLoader(entry.getValue()), Ints.checkedCast(streamBufferSize.toBytes())); slices.put(entry.getKey(), sliceInput); } return slices.build(); }
@Override public void readBytes(byte[] destination, int destinationIndex, int length) { checkBound(position() + length, globalLength, "End of stream"); while (length > 0) { int bytesToRead = min(available(), length); buffer.getBytes(bufferPosition, destination, destinationIndex, bytesToRead); bufferPosition += bytesToRead; length -= bytesToRead; destinationIndex += bytesToRead; ensureAvailable(min(length, buffer.length())); } }
@Override public int read(byte[] destination, int destinationIndex, int length) { if (length == 0) { return 0; } if (globalLength - position() == 0) { return -1; } // limit read to stream size length = (int) min(length, globalLength - position()); // do a full read of the available data readBytes(destination, destinationIndex, length); return length; }
@Override public int readUnsignedByte() { return readByte() & 0xFF; }