/** * Deserialize the index into a structure and return it * * @param in input source * @param type the comparator type for the column family * * @return ArrayList<IndexInfo> - list of de-serialized indexes * @throws IOException if an I/O error occurs. */ public static List<IndexInfo> deserializeIndex(FileDataInput in, CType type) throws IOException { int columnIndexSize = in.readInt(); if (columnIndexSize == 0) return Collections.<IndexInfo>emptyList(); ArrayList<IndexInfo> indexList = new ArrayList<IndexInfo>(); FileMark mark = in.mark(); ISerializer<IndexInfo> serializer = type.indexSerializer(); while (in.bytesPastMark(mark) < columnIndexSize) { indexList.add(serializer.deserialize(in)); } assert in.bytesPastMark(mark) == columnIndexSize; return indexList; }
public void close() throws IOException { if (shouldCloseFile && file != null) file.close(); } }
public static CorruptColumnException create(DataInput in, ByteBuffer name) { assert name.remaining() <= 0; String format = "invalid column name length %d%s"; String details = ""; if (in instanceof FileDataInput) { FileDataInput fdis = (FileDataInput)in; long remaining; try { remaining = fdis.bytesRemaining(); } catch (IOException e) { throw new FSReadError(e, fdis.getPath()); } details = String.format(" (%s, %d bytes remaining)", fdis.getPath(), remaining); } return new CorruptColumnException(String.format(format, name.remaining(), details)); } }
public static long calculateClaimedChecksum(FileDataInput input, int commitLogVersion) throws IOException { switch (commitLogVersion) { case CommitLogDescriptor.VERSION_12: case CommitLogDescriptor.VERSION_20: return input.readLong(); // Changed format in 2.1 default: return input.readInt() & 0xffffffffL; } }
IndexInfo fetchIndex(int index) throws IOException { retrievals++; // seek to position in "offsets to IndexInfo" table indexReader.seek(indexInfoFilePosition + offsetsOffset + index * TypeSizes.sizeof(0)); // read offset of IndexInfo int indexInfoPos = indexReader.readInt(); // seek to posision of IndexInfo indexReader.seek(indexInfoFilePosition + indexInfoPos); // finally, deserialize IndexInfo return idxInfoSerializer.deserialize(indexReader); } }
if (desc.id == minPosition.segmentId && reader.getFilePointer() < minPosition.position) reader.seek(minPosition.position); while (statusTracker.shouldContinue() && reader.getFilePointer() < end && !reader.isEOF()) long mutationStart = reader.getFilePointer(); if (logger.isTraceEnabled()) logger.trace("Reading mutation at {}", mutationStart); if(end - reader.getFilePointer() < 4) serializedSize = reader.readInt(); if (serializedSize == LEGACY_END_OF_SEGMENT_MARKER) logger.trace("Encountered end of segment marker at {}", reader.getFilePointer()); statusTracker.requestTermination(); return; reader.readFully(buffer, 0, serializedSize); long mutationPosition = reader.getFilePointer(); readMutation(handler, buffer, serializedSize, minPosition, (int)mutationPosition, desc);
int flags = in.readUnsignedByte(); if (isEndOfPartition(flags)) return null; boolean deletionIsShadowable = (extendedFlags & HAS_SHADOWABLE_DELETION) != 0; Clustering clustering = Clustering.serializer.deserialize(in, helper.version, header.clusteringTypes()); long nextPosition = in.readUnsignedVInt() + in.getFilePointer(); in.readUnsignedVInt(); // skip previous unfiltered size if (hasTimestamp) in.seek(nextPosition); return BTreeRow.emptyDeletedRow(clustering, deletion);
try (FileDataInput in = ifile.createReader(sampledPosition)) path = in.getPath(); while (!in.isEOF()) RowIndexEntry indexEntry = rowIndexEntrySerializer.deserialize(in, in.getFilePointer()); if (exactMatch && updateCacheAndStats) throw new AssertionError(String.format("%s != %s in %s", keyInDisk, key, fdi.getPath()));
file.seek(positionToSeek); FileMark mark = file.mark(); while (file.bytesPastMark(mark) < indexInfo.width && nextToFetch != null)
public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInput dfile, RowIndexEntry<?> indexEntry, DecoratedKey key, boolean tombstoneOnly) { try { dfile.seek(indexEntry.position); ByteBufferUtil.skipShortLength(dfile); // Skip partition key DeletionTime partitionLevelDeletion = DeletionTime.serializer.deserialize(dfile); SerializationHelper helper = new SerializationHelper(sstable.metadata, sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL); SSTableSimpleIterator iterator = tombstoneOnly ? SSTableSimpleIterator.createTombstoneOnly(sstable.metadata, dfile, sstable.header, helper, partitionLevelDeletion) : SSTableSimpleIterator.create(sstable.metadata, dfile, sstable.header, helper, partitionLevelDeletion); return new SSTableIdentityIterator(sstable, key, partitionLevelDeletion, dfile.getPath(), iterator); } catch (IOException e) { sstable.markSuspect(); throw new CorruptSSTableException(e, dfile.getPath()); } }
public void close() { try { closeInternal(); } catch (IOException e) { sstable.markSuspect(); throw new CorruptSSTableException(e, reader.file.getPath()); } }
long currentFilePointer = reader.file.getFilePointer(); if (startOfBlock == currentFilePointer) mark = reader.file.mark(); mark = reader.file.mark(); reader.seekToPosition(currentFilePointer);
public DecoratedKey keyAt(long indexPosition) throws IOException { DecoratedKey key; try (FileDataInput in = ifile.createReader(indexPosition)) { if (in.isEOF()) return null; key = decorateKey(ByteBufferUtil.readWithShortLength(in)); // hint read path about key location if caching is enabled // this saves index summary lookup and index file iteration which whould be pretty costly // especially in presence of promoted column indexes if (isKeyCacheSetup()) cacheKey(key, rowIndexEntrySerializer.deserialize(in, in.getFilePointer())); } return key; }
IndexInfo fetchIndex(int index) throws IOException { retrievals++; // seek to posision of IndexInfo indexReader.seek(indexInfoFilePosition + offsets[index]); // deserialize IndexInfo return idxInfoSerializer.deserialize(indexReader); } }
public boolean isPastCurrentBlock() throws IOException { assert reader.deserializer != null; long correction = reader.deserializer.bytesReadForUnconsumedData(); return reader.file.bytesPastMark(mark) - correction >= currentIndex().width; }
public boolean isOpen() { try { return fileDataInput.isEOF(); } catch (IOException e) { return true; } }
public void setToBlock(int blockIdx) throws IOException { if (blockIdx >= 0 && blockIdx < indexEntry.columnsIndexCount()) { reader.seekToPosition(columnOffset(blockIdx)); reader.deserializer.clearState(); } currentIndexIdx = blockIdx; reader.openMarker = blockIdx > 0 ? index(blockIdx - 1).endOpenMarker : null; mark = reader.file.mark(); }