50   fCompressor = std::make_unique<RNTupleCompressor>();
 
 
   92   fCounters->fNPageCommitted.Inc();
 
   93   fCounters->fSzWritePayload.Add(
sealedPage.GetBufferSize());
 
   94   fNBytesCurrentCluster += 
sealedPage.GetBufferSize();
 
 
  108   fCounters->fSzZip.Add(
page.GetNBytes());
 
 
  116   const auto nBits = fDescriptorBuilder.GetDescriptor().GetColumnDescriptor(
physicalColumnId).GetBitsOnStorage();
 
 
  122                                                                     std::vector<RNTupleLocator> &
locators)
 
  126   std::uint64_t 
offset = fWriter->ReserveBlob(
batch.fSize, 
batch.fBytesPacked);
 
  139   fCounters->fNPageCommitted.Add(
batch.fSealedPages.size());
 
  140   fCounters->fSzWritePayload.Add(
batch.fSize);
 
  141   fNBytesCurrentCluster += 
batch.fSize;
 
  144   batch.fBytesPacked = 0;
 
  145   batch.fSealedPages.clear();
 
 
  148std::vector<ROOT::Experimental::RNTupleLocator>
 
  150                                                                   const std::vector<bool> &
mask)
 
  152   const std::uint64_t 
maxKeySize = fOptions->GetMaxKeySize();
 
  155   std::vector<ROOT::Experimental::RNTupleLocator> 
locators;
 
  157   std::size_t 
iPage = 0;
 
  166         fDescriptorBuilder.GetDescriptor().GetColumnDescriptor(
range.fPhysicalColumnId).GetBitsOnStorage();
 
  200            fCounters->fNPageCommitted.Inc();
 
  201            fCounters->fSzWritePayload.Add(
sealedPageIt->GetBufferSize());
 
  212   if (
batch.fSize > 0) {
 
 
  221   auto result = fNBytesCurrentCluster;
 
  222   fNBytesCurrentCluster = 0;
 
 
  243   fWriter->UpdateStreamerInfos(fDescriptorBuilder.BuildStreamerInfos());
 
 
  256     fClusterPool(std::make_unique<
RClusterPool>(*
this, options.GetClusterBunchSize()))
 
 
  262                                                               std::unique_ptr<ROOT::Internal::RRawFile> file,
 
  266   fFile = std::move(file);
 
 
  277std::unique_ptr<ROOT::Experimental::Internal::RPageSourceFile>
 
  282      throw RException(
R__FAIL(
"This RNTuple object was not streamed from a ROOT file (TFile or descendant)"));
 
  284   std::unique_ptr<ROOT::Internal::RRawFile> 
rawFile;
 
  288   std::string className = 
anchor.fFile->IsA()->GetName();
 
  289   auto url = 
anchor.fFile->GetEndpointUrl();
 
  290   auto protocol = std::string(
url->GetProtocol());
 
  291   if (className == 
"TFile") {
 
  293   } 
else if (className == 
"TDavixFile" || className == 
"TNetXNGFile") {
 
  299   auto pageSource = std::make_unique<RPageSourceFile>(
"", std::move(
rawFile), options);
 
 
  312      fAnchor = fReader.GetNTuple(fNTupleName).Unwrap();
 
  314   fReader.SetMaxKeySize(fAnchor->GetMaxKeySize());
 
  318      throw RException(
R__FAIL(
"unsupported RNTuple epoch version: " + std::to_string(fAnchor->GetVersionEpoch())));
 
  321   fDescriptorBuilder.SetOnDiskHeaderSize(fAnchor->GetNBytesHeader());
 
  322   fDescriptorBuilder.AddToOnDiskFooterSize(fAnchor->GetNBytesFooter());
 
  325   const auto bufSize = fAnchor->GetNBytesHeader() + fAnchor->GetNBytesFooter() +
 
  326                        std::max(fAnchor->GetLenHeader(), fAnchor->GetLenFooter());
 
  327   fStructureBuffer.fBuffer = std::make_unique<unsigned char[]>(
bufSize);
 
  328   fStructureBuffer.fPtrHeader = fStructureBuffer.fBuffer.get();
 
  329   fStructureBuffer.fPtrFooter = fStructureBuffer.fBuffer.get() + fAnchor->GetNBytesHeader();
 
  336       (std::max(fAnchor->GetNBytesHeader(), fAnchor->GetNBytesFooter()) > 
readvLimits.fMaxSingleSize) ||
 
  337       (fAnchor->GetNBytesHeader() + fAnchor->GetNBytesFooter() > 
readvLimits.fMaxTotalSize)) {
 
  339      fReader.ReadBuffer(fStructureBuffer.fPtrHeader, fAnchor->GetNBytesHeader(), fAnchor->GetSeekHeader());
 
  340      fReader.ReadBuffer(fStructureBuffer.fPtrFooter, fAnchor->GetNBytesFooter(), fAnchor->GetSeekFooter());
 
  341      fCounters->fNRead.Add(2);
 
  345         {fStructureBuffer.fPtrHeader, fAnchor->GetSeekHeader(), fAnchor->GetNBytesHeader(), 0},
 
  346         {fStructureBuffer.fPtrFooter, fAnchor->GetSeekFooter(), fAnchor->GetNBytesFooter(), 0}};
 
  348      fCounters->fNReadV.Inc();
 
 
  354   auto unzipBuf = 
reinterpret_cast<unsigned char *
>(fStructureBuffer.fPtrFooter) + fAnchor->GetNBytesFooter();
 
  364   auto desc = fDescriptorBuilder.MoveDescriptor();
 
  366   std::vector<unsigned char> buffer;
 
  367   for (
const auto &
cgDesc : desc.GetClusterGroupIterable()) {
 
  369         std::max<size_t>(buffer.size(), 
cgDesc.GetPageListLength() + 
cgDesc.GetPageListLocator().fBytesOnStorage));
 
  371      fReader.ReadBuffer(
zipBuffer, 
cgDesc.GetPageListLocator().fBytesOnStorage,
 
  372                         cgDesc.GetPageListLocator().GetPosition<std::uint64_t>());
 
  380   fFile->SetBuffering(
false);
 
 
  404                         pageInfo.fLocator.GetPosition<std::uint64_t>());
 
  410   sealedPage.VerifyChecksumIfEnabled().ThrowOnError();
 
 
  445                            pageInfo.fLocator.GetPosition<std::uint64_t>());
 
  447      fCounters->fNPageRead.Inc();
 
  448      fCounters->fNRead.Inc();
 
  449      fCounters->fSzReadPayload.Add(
sealedPage.GetBufferSize());
 
  452      if (!fCurrentCluster || (fCurrentCluster->GetId() != 
clusterId) || !fCurrentCluster->ContainsColumn(
columnId))
 
  453         fCurrentCluster = fClusterPool->GetCluster(
clusterId, fActivePhysicalColumns.ToColumnSet());
 
  461      auto onDiskPage = fCurrentCluster->GetOnDiskPage(key);
 
  475   fCounters->fNPageUnsealed.Inc();
 
 
  479std::unique_ptr<ROOT::Experimental::Internal::RPageSource>
 
  483   clone->fFile = fFile->Clone();
 
  485   return std::unique_ptr<RPageSourceFile>(clone);
 
 
  488std::unique_ptr<ROOT::Experimental::Internal::RCluster>
 
  495      std::uint64_t fOffset = 0;
 
  496      std::uint64_t 
fSize = 0;
 
  502   auto pageZeroMap = std::make_unique<ROnDiskPageMap>();
 
  527   std::vector<std::size_t> 
gaps;
 
  530   for (
unsigned i = 1; i < 
onDiskPages.size(); ++i) {
 
  533      gaps.emplace_back(std::max(gap, std::int64_t(0)));
 
  541   for (
auto g : 
gaps) {
 
  563   const std::uint64_t 
maxKeySize = fReader.GetMaxKeySize();
 
  568      const std::uint64_t 
overhead = std::max(
static_cast<std::int64_t
>(s.fOffset) - 
readUpTo, std::int64_t(0));
 
  569      const std::uint64_t 
extent = std::max(
static_cast<std::int64_t
>(s.fOffset + s.fSize) - 
readUpTo, std::int64_t(0));
 
  573         s.fBufPos = 
reinterpret_cast<intptr_t
>(req.
fBuffer) + s.fOffset - req.
fOffset;
 
  583      s.fBufPos = 
reinterpret_cast<intptr_t
>(req.
fBuffer);
 
  590   fCounters->fSzReadPayload.Add(
szPayload);
 
  594   auto buffer = 
new unsigned char[
reinterpret_cast<intptr_t
>(req.
fBuffer) + req.
fSize];
 
  595   auto pageMap = std::make_unique<ROnDiskPageMapHeap>(std::unique_ptr<
unsigned char[]>(buffer));
 
 
  613std::vector<std::unique_ptr<ROOT::Experimental::Internal::RCluster>>
 
  616   fCounters->fNClusterLoaded.Add(
clusterKeys.size());
 
  618   std::vector<std::unique_ptr<ROOT::Experimental::Internal::RCluster>> 
clusters;
 
  619   std::vector<ROOT::Internal::RRawFile::RIOVec> 
readRequests;
 
  637         for (std::size_t i = 0; i < 
nBatch; ++i) {
 
  659      fCounters->fNReadV.Inc();
 
  660      fCounters->fNRead.Add(
nBatch);
 
 
#define R__FAIL(msg)
Short-hand to return an RResult<T> in an error state; the RError is implicitly converted into RResult...
ROOT::Detail::TRangeCast< T, true > TRangeDynCast
TRangeDynCast is an adapter class that allows the typed iteration through a TCollection.
#define R__ASSERT(e)
Checks condition e and reports a fatal error if it's false.
Option_t Option_t TPoint TPoint const char GetTextMagnitude GetFillStyle GetLineColor GetLineWidth GetMarkerStyle GetTextAlign GetTextColor GetTextSize void char Point_t Rectangle_t WindowAttributes_t Float_t Float_t Float_t Int_t Int_t UInt_t UInt_t Rectangle_t mask
Option_t Option_t TPoint TPoint const char GetTextMagnitude GetFillStyle GetLineColor GetLineWidth GetMarkerStyle GetTextAlign GetTextColor GetTextSize void char Point_t Rectangle_t WindowAttributes_t Float_t Float_t Float_t Int_t Int_t UInt_t UInt_t Rectangle_t Int_t Int_t Window_t TString Int_t GCValues_t GetPrimarySelectionOwner GetDisplay GetScreen GetColormap GetNativeEvent const char const char dpyName wid window const char font_name cursor keysym reg const char only_if_exist regb h Point_t winding char text const char depth char const char Int_t count const char ColorStruct_t color const char Pixmap_t Pixmap_t PictureAttributes_t attr const char char ret_data h unsigned char height h offset
Option_t Option_t TPoint TPoint const char GetTextMagnitude GetFillStyle GetLineColor GetLineWidth GetMarkerStyle GetTextAlign GetTextColor GetTextSize void char Point_t Rectangle_t WindowAttributes_t Float_t Float_t Float_t Int_t Int_t UInt_t UInt_t Rectangle_t result
Option_t Option_t TPoint TPoint const char GetTextMagnitude GetFillStyle GetLineColor GetLineWidth GetMarkerStyle GetTextAlign GetTextColor GetTextSize void char Point_t Rectangle_t WindowAttributes_t Float_t Float_t Float_t Int_t Int_t UInt_t UInt_t Rectangle_t Int_t Int_t Window_t TString Int_t GCValues_t GetPrimarySelectionOwner GetDisplay GetScreen GetColormap GetNativeEvent const char const char dpyName wid window const char font_name cursor keysym reg const char only_if_exist regb h Point_t winding char text const char depth char const char Int_t count const char ColorStruct_t color const char Pixmap_t Pixmap_t PictureAttributes_t attr const char char ret_data h unsigned char height h length
Managed a set of clusters containing compressed and packed pages.
Read RNTuple data blocks from a TFile container, provided by a RRawFile.
static Writer_t MakeMemCopyWriter(unsigned char *dest)
static void Unzip(const void *from, size_t nbytes, size_t dataLen, void *to)
The nbytes parameter provides the size ls of the from buffer.
static std::unique_ptr< RNTupleFileWriter > Recreate(std::string_view ntupleName, std::string_view path, EContainerFormat containerFormat, const RNTupleWriteOptions &options)
Create or truncate the local file given by path with the new empty RNTuple identified by ntupleName.
static std::unique_ptr< RNTupleFileWriter > Append(std::string_view ntupleName, TDirectory &fileOrDirectory, std::uint64_t maxKeySize)
The directory parameter can also be a TFile object (TFile inherits from TDirectory).
static RResult< void > DeserializeHeader(const void *buffer, std::uint64_t bufSize, RNTupleDescriptorBuilder &descBuilder)
static RResult< void > DeserializeFooter(const void *buffer, std::uint64_t bufSize, RNTupleDescriptorBuilder &descBuilder)
static RResult< void > DeserializePageList(const void *buffer, std::uint64_t bufSize, DescriptorId_t clusterGroupId, RNTupleDescriptor &desc)
A page as being stored on disk, that is packed and compressed.
Base class for a sink with a physical storage backend.
void EnableDefaultMetrics(const std::string &prefix)
Enables the default set of metrics provided by RPageSink.
Storage provider that write ntuple pages into a file.
std::vector< RNTupleLocator > CommitSealedPageVImpl(std::span< RPageStorage::RSealedPageGroup > ranges, const std::vector< bool > &mask) final
Vector commit of preprocessed pages.
RNTupleLocator WriteSealedPage(const RPageStorage::RSealedPage &sealedPage, std::size_t bytesPacked)
We pass bytesPacked so that TFile::ls() reports a reasonable value for the compression ratio of the c...
void InitImpl(unsigned char *serializedHeader, std::uint32_t length) final
std::uint64_t StageClusterImpl() final
Returns the number of bytes written to storage (excluding metadata)
RNTupleLocator CommitSealedPageImpl(DescriptorId_t physicalColumnId, const RPageStorage::RSealedPage &sealedPage) final
RNTupleLocator CommitClusterGroupImpl(unsigned char *serializedPageList, std::uint32_t length) final
Returns the locator of the page list envelope of the given buffer that contains the serialized page l...
~RPageSinkFile() override
void CommitBatchOfPages(CommitBatch &batch, std::vector< RNTupleLocator > &locators)
Subroutine of CommitSealedPageVImpl, used to perform a vector write of the (multi-)range of pages con...
RPageSinkFile(std::string_view ntupleName, const RNTupleWriteOptions &options)
std::unique_ptr< RNTupleFileWriter > fWriter
RNTupleLocator CommitPageImpl(ColumnHandle_t columnHandle, const RPage &page) override
void CommitDatasetImpl() final
std::unique_ptr< RNTupleCompressor > fCompressor
Helper to zip pages and header/footer; includes a 16MB (kMAXZIPBUF) zip buffer.
Storage provider that reads ntuple pages from a file.
static std::unique_ptr< RPageSourceFile > CreateFromAnchor(const RNTuple &anchor, const RNTupleReadOptions &options=RNTupleReadOptions())
Used from the RNTuple class to build a datasource if the anchor is already available.
RPageRef LoadPageImpl(ColumnHandle_t columnHandle, const RClusterInfo &clusterInfo, ClusterSize_t::ValueType idxInCluster) final
RPageSourceFile(std::string_view ntupleName, const RNTupleReadOptions &options)
std::vector< std::unique_ptr< RCluster > > LoadClusters(std::span< RCluster::RKey > clusterKeys) final
Populates all the pages of the given cluster ids and columns; it is possible that some columns do not...
std::unique_ptr< RPageSource > CloneImpl() const final
The cloned page source creates a new raw file and reader and opens its own file descriptor to the dat...
std::unique_ptr< RCluster > PrepareSingleCluster(const RCluster::RKey &clusterKey, std::vector< ROOT::Internal::RRawFile::RIOVec > &readRequests)
Helper function for LoadClusters: it prepares the memory buffer (page map) and the read requests for ...
RMiniFileReader fReader
Takes the fFile to read ntuple blobs from it.
void LoadSealedPage(DescriptorId_t physicalColumnId, RClusterIndex clusterIndex, RSealedPage &sealedPage) final
Read the packed and compressed bytes of a page into the memory buffer provided by sealedPage.
~RPageSourceFile() override
void LoadStructureImpl() final
RNTupleDescriptor AttachImpl() final
LoadStructureImpl() has been called before AttachImpl() is called
std::unique_ptr< ROOT::Internal::RRawFile > fFile
An RRawFile is used to request the necessary byte ranges from a local or a remote file.
Abstract interface to read data from an ntuple.
void EnableDefaultMetrics(const std::string &prefix)
Enables the default set of metrics provided by RPageSource.
Stores information about the cluster in which this page resides.
A page is a slice of a column that is mapped into memory.
static RPage MakePageZero(ColumnId_t columnId, ClusterSize_t::ValueType elementSize)
Make a 'zero' page for column columnId (that is comprised of 0x00 bytes only).
static const void * GetPageZeroBuffer()
Return a pointer to the page zero buffer used if there is no on-disk data for a particular deferred c...
Addresses a column element or field item relative to a particular cluster, instead of a global NTuple...
Base class for all ROOT issued exceptions.
The on-storage meta-data of an ntuple.
Common user-tunable settings for reading ntuples.
Common user-tunable settings for storing ntuples.
std::uint64_t GetMaxKeySize() const
The RRawFileTFile wraps an open TFile, but does not take ownership.
The RRawFile provides read-only access to local and remote files.
static std::unique_ptr< RRawFile > Create(std::string_view url, ROptions options=ROptions())
Factory method that returns a suitable concrete implementation according to the transport in the url.
Representation of an RNTuple data set in a ROOT file.
static constexpr std::uint16_t kVersionEpoch
const_iterator begin() const
const_iterator end() const
Describe directory structure in memory.
std::uint64_t NTupleSize_t
Integer type long enough to hold the maximum number of entries in a column.
std::uint64_t DescriptorId_t
Distriniguishes elements of the same type within a descriptor, e.g. different fields.
tbb::task_arena is an alias of tbb::interface7::task_arena, which doesn't allow to forward declare tb...
The identifiers that specifies the content of a (partial) cluster.
On-disk pages within a page source are identified by the column and page number.
Summarizes cluster-level information that are necessary to load a certain page.
A sealed page contains the bytes of a page as written to storage (packed & compressed).
Generic information about the physical location of data.
Used for vector reads from multiple offsets into multiple buffers.
std::size_t fSize
The number of desired bytes.
void * fBuffer
The destination for reading.
std::uint64_t fOffset
The file offset.