AvroIO.java org::apache::iceberg::avro::AvroIO org::apache::iceberg::avro::AvroIO::AvroInputStreamAdapter org::apache::iceberg::avro /* *LicensedtotheApacheSoftwareFoundation(ASF)underone *ormorecontributorlicenseagreements.SeetheNOTICEfile *distributedwiththisworkforadditionalinformation *regardingcopyrightownership.TheASFlicensesthisfile *toyouundertheApacheLicense,Version2.0(the *"License");youmaynotusethisfileexceptincompliance *withtheLicense.YoumayobtainacopyoftheLicenseat * *http://www.apache.org/licenses/LICENSE-2.0 * *Unlessrequiredbyapplicablelaworagreedtoinwriting, *softwaredistributedundertheLicenseisdistributedonan *"ASIS"BASIS,WITHOUTWARRANTIESORCONDITIONSOFANY *KIND,eitherexpressorimplied.SeetheLicenseforthe *specificlanguagegoverningpermissionsandlimitations *undertheLicense. */ packageorg.apache.iceberg.avro; importjava.io.EOFException; importjava.io.IOException; importjava.io.InputStream; importjava.util.Arrays; importjava.util.Map; importjava.util.function.Supplier; importorg.apache.avro.InvalidAvroMagicException; importorg.apache.avro.file.SeekableInput; importorg.apache.avro.io.BinaryDecoder; importorg.apache.avro.io.DecoderFactory; importorg.apache.iceberg.common.DynClasses; importorg.apache.iceberg.common.DynConstructors; importorg.apache.iceberg.exceptions.RuntimeIOException; importorg.apache.iceberg.io.DelegatingInputStream; importorg.apache.iceberg.io.SeekableInputStream; classAvroIO{ privatestaticfinalbyte[]AVRO_MAGIC=newbyte[]{'O','b','j',1}; privatestaticfinalValueReader<byte[]>MAGIC_READER=ValueReaders.fixed(AVRO_MAGIC.length); privatestaticfinalValueReader<Map<String,String>>META_READER= ValueReaders.map(ValueReaders.strings(),ValueReaders.strings()); privatestaticfinalValueReader<byte[]>SYNC_READER=ValueReaders.fixed(16); privateAvroIO(){} privatestaticfinalClass<?>fsDataInputStreamClass= DynClasses.builder().impl("org.apache.hadoop.fs.FSDataInputStream").orNull().build(); privatestaticfinalbooleanrelocated= "org.apache.avro.file.SeekableInput".equals(SeekableInput.class.getName()); privatestaticfinalDynConstructors.Ctor<SeekableInput>avroFsInputCtor= !relocated&&fsDataInputStreamClass!=null ?DynConstructors.builder(SeekableInput.class) .impl("org.apache.hadoop.fs.AvroFSInput",fsDataInputStreamClass,Long.TYPE) .build() :null; staticSeekableInputstream(SeekableInputStreamstream,longlength){ if(streaminstanceofDelegatingInputStream){ InputStreamwrapped=((DelegatingInputStream)stream).getDelegate(); if(avroFsInputCtor!=null &&fsDataInputStreamClass!=null &&fsDataInputStreamClass.isInstance(wrapped)){ returnavroFsInputCtor.newInstance(wrapped,length); } } returnnewAvroInputStreamAdapter(stream,length); } privatestaticclassAvroInputStreamAdapterextendsSeekableInputStreamimplementsSeekableInput{ privatefinalSeekableInputStreamstream; privatefinallonglength; AvroInputStreamAdapter(SeekableInputStreamstream,longlength){ this.stream=stream; this.length=length; } @Override publicvoidclose()throwsIOException{ stream.close(); } @Override publiclonggetPos()throwsIOException{ returnstream.getPos(); } @Override publicvoidseek(longnewPos)throwsIOException{ stream.seek(newPos); } @Override publiclongtell()throwsIOException{ returngetPos(); } @Override publiclonglength()throwsIOException{ returnlength; } @Override publicintread()throwsIOException{ returnstream.read(); } @Override publicintread(byte[]b)throwsIOException{ returnstream.read(b); } @Override publicintread(byte[]b,intoff,intlen)throwsIOException{ returnstream.read(b,off,len); } @Override publiclongskip(longn)throwsIOException{ returnstream.skip(n); } @Override publicintavailable()throwsIOException{ returnstream.available(); } @Override publicsynchronizedvoidmark(intreadlimit){ stream.mark(readlimit); } @Override publicsynchronizedvoidreset()throwsIOException{ stream.reset(); } @Override publicbooleanmarkSupported(){ returnstream.markSupported(); } } staticlongfindStartingRowPos(Supplier<SeekableInputStream>open,longstart){ longtotalRows=0; try(SeekableInputStreamin=open.get()){ //useadirectdecoderthatwillnotbuffersothepositionoftheinputstreamisaccurate BinaryDecoderdecoder=DecoderFactory.get().directBinaryDecoder(in,null); //anAvrofile'slayoutlookslikethis: //header|block|block|... //theheadercontains: //magic|string-map|sync //eachblockconsistsof: //row-count|compressed-size-in-bytes|block-bytes|sync //itisnecessarytoreadtheheaderherebecausethisistheonlywaytogettheexpected //filesyncbytes byte[]magic=MAGIC_READER.read(decoder,null); if(!Arrays.equals(AVRO_MAGIC,magic)){ thrownewInvalidAvroMagicException("NotanAvrofile"); } META_READER.read(decoder,null);//ignorethefilemetadata,itisn'tneeded byte[]fileSync=SYNC_READER.read(decoder,null); //thewhileloopreadsrowcountsandseekspasttheblockbytesuntilthenextsyncposis //>=start,which //indicatesthatthenextsyncisthestartofthesplit. byte[]blockSync=newbyte[16]; longnextSyncPos=in.getPos(); while(nextSyncPos<start){ if(nextSyncPos!=in.getPos()){ in.seek(nextSyncPos); SYNC_READER.read(decoder,blockSync); if(!Arrays.equals(fileSync,blockSync)){ thrownewRuntimeIOException("Invalidsyncat%s",nextSyncPos); } } longrowCount=decoder.readLong(); longcompressedBlockSize=decoder.readLong(); totalRows+=rowCount; nextSyncPos=in.getPos()+compressedBlockSize; } returntotalRows; }catch(EOFExceptione){ returntotalRows; }catch(IOExceptione){ thrownewRuntimeIOException(e,"Failedtoreadstreamwhilefindingstartingrowposition"); } } }