From 6f08301ba95297f54026c53aeb902e0d67010bfa Mon Sep 17 00:00:00 2001 From: Podhorszki Norbert Date: Thu, 1 Jul 2021 12:57:19 -0400 Subject: [PATCH 01/18] align writers to a page size (set to 64KB for now) --- examples/basics/globalArray/globalArray_write.cpp | 2 ++ source/adios2/engine/bp5/BP5Writer.cpp | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/examples/basics/globalArray/globalArray_write.cpp b/examples/basics/globalArray/globalArray_write.cpp index 2b88a04960..b53d01d9ae 100644 --- a/examples/basics/globalArray/globalArray_write.cpp +++ b/examples/basics/globalArray/globalArray_write.cpp @@ -63,6 +63,8 @@ int main(int argc, char *argv[]) // Get io settings from the config file or // create one with default settings here adios2::IO io = adios.DeclareIO("Output"); + io.SetEngine("BP5"); + io.SetParameter("NumAggregators", "1"); /* * Define global array: type, name, global dimensions diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index aab245698c..18f59fc5d9 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -131,6 +131,7 @@ uint64_t BP5Writer::WriteMetadata( return MetaDataSize; } +static const uint64_t PAGE_SIZE = 65536; // 64KB void BP5Writer::WriteData(format::BufferV *Data) { format::BufferV::BufferV_iovec DataVec = Data->DataVec(); @@ -154,6 +155,8 @@ void BP5Writer::WriteData(format::BufferV *Data) nextWriterPos += DataVec[i].iov_len; i++; } + // align to PAGE_SIZE + nextWriterPos += PAGE_SIZE - (nextWriterPos % PAGE_SIZE); m_Aggregator.m_Comm.Isend(&nextWriterPos, 1, m_Aggregator.m_Comm.Rank() + 1, 0, "Chain token in BP5Writer::WriteData"); @@ -164,6 +167,8 @@ void BP5Writer::WriteData(format::BufferV *Data) { if (i == 0) { + std::cout << "Rank " << m_Comm.Rank() + << " write to position = " << m_StartDataPos << std::endl; m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, DataVec[i].iov_len, m_StartDataPos); } @@ -182,6 +187,8 @@ void BP5Writer::WriteData(format::BufferV *Data) // so it can update its data pos if (m_Aggregator.m_Comm.Rank() == m_Aggregator.m_Comm.Size() - 1) { + // align to PAGE_SIZE + m_DataPos += PAGE_SIZE - (m_DataPos % PAGE_SIZE); m_Aggregator.m_Comm.Isend( &m_DataPos, 1, 0, 0, "Final chain token in BP5Writer::WriteData"); From 6f3ef17ad9edaed8a272e3cf8740c6785c6bf3b6 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Fri, 2 Jul 2021 08:06:57 -0400 Subject: [PATCH 02/18] remove printf --- source/adios2/engine/bp5/BP5Writer.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index 18f59fc5d9..0f77952d24 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -167,8 +167,6 @@ void BP5Writer::WriteData(format::BufferV *Data) { if (i == 0) { - std::cout << "Rank " << m_Comm.Rank() - << " write to position = " << m_StartDataPos << std::endl; m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, DataVec[i].iov_len, m_StartDataPos); } From d663bf435411d55a295ef28f46814154aeb1747b Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Mon, 12 Jul 2021 09:57:54 -0400 Subject: [PATCH 03/18] Add MPIShmChain aggregator class to create a two-level aggregator chain. Number of aggregators >= Number of substreams. An aggregator shares memory with all its members (level 1). Multiple aggregator chains can be assigned to one substream (subfile index) (level 2). Two level aggregation is not implemented yet. --- source/adios2/CMakeLists.txt | 3 +- source/adios2/engine/bp3/BP3Writer.cpp | 1 + source/adios2/engine/bp4/BP4Writer.cpp | 1 + source/adios2/engine/bp5/BP5Engine.cpp | 29 ++- source/adios2/engine/bp5/BP5Engine.h | 9 + source/adios2/engine/bp5/BP5Writer.cpp | 55 ++++- source/adios2/engine/bp5/BP5Writer.h | 19 +- .../engine/bp5/BP5Writer_TwoLevelShm.cpp | 128 +++++++++++ .../toolkit/aggregator/mpi/MPIAggregator.cpp | 5 +- .../toolkit/aggregator/mpi/MPIAggregator.h | 8 +- .../toolkit/aggregator/mpi/MPIChain.cpp | 5 +- .../adios2/toolkit/aggregator/mpi/MPIChain.h | 4 +- .../toolkit/aggregator/mpi/MPIShmChain.cpp | 208 ++++++++++++++++++ .../toolkit/aggregator/mpi/MPIShmChain.h | 104 +++++++++ 14 files changed, 554 insertions(+), 25 deletions(-) create mode 100644 source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp create mode 100644 source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp create mode 100644 source/adios2/toolkit/aggregator/mpi/MPIShmChain.h diff --git a/source/adios2/CMakeLists.txt b/source/adios2/CMakeLists.txt index 5168f47fc3..ac036822c9 100644 --- a/source/adios2/CMakeLists.txt +++ b/source/adios2/CMakeLists.txt @@ -99,6 +99,7 @@ add_library(adios2_core toolkit/aggregator/mpi/MPIAggregator.cpp toolkit/aggregator/mpi/MPIChain.cpp + toolkit/aggregator/mpi/MPIShmChain.cpp toolkit/burstbuffer/FileDrainer.cpp toolkit/burstbuffer/FileDrainerSingleThread.cpp @@ -124,7 +125,7 @@ if (ADIOS2_HAVE_BP5) target_sources(adios2_core PRIVATE engine/bp5/BP5Engine.cpp engine/bp5/BP5Reader.cpp engine/bp5/BP5Reader.tcc - engine/bp5/BP5Writer.cpp engine/bp5/BP5Writer.tcc + engine/bp5/BP5Writer.cpp engine/bp5/BP5Writer.tcc engine/bp5/BP5Writer_TwoLevelShm.cpp ) endif() diff --git a/source/adios2/engine/bp3/BP3Writer.cpp b/source/adios2/engine/bp3/BP3Writer.cpp index bf2aa9afc7..4cd48cdfdb 100644 --- a/source/adios2/engine/bp3/BP3Writer.cpp +++ b/source/adios2/engine/bp3/BP3Writer.cpp @@ -123,6 +123,7 @@ void BP3Writer::Init() static_cast(m_BP3Serializer.m_SizeMPI)) { m_BP3Serializer.m_Aggregator.Init( + m_BP3Serializer.m_Parameters.NumAggregators, m_BP3Serializer.m_Parameters.NumAggregators, m_Comm); } InitTransports(); diff --git a/source/adios2/engine/bp4/BP4Writer.cpp b/source/adios2/engine/bp4/BP4Writer.cpp index ba9fa834db..1e4a2b016c 100644 --- a/source/adios2/engine/bp4/BP4Writer.cpp +++ b/source/adios2/engine/bp4/BP4Writer.cpp @@ -127,6 +127,7 @@ void BP4Writer::Init() static_cast(m_BP4Serializer.m_SizeMPI)) { m_BP4Serializer.m_Aggregator.Init( + m_BP4Serializer.m_Parameters.NumAggregators, m_BP4Serializer.m_Parameters.NumAggregators, m_Comm); } InitTransports(); diff --git a/source/adios2/engine/bp5/BP5Engine.cpp b/source/adios2/engine/bp5/BP5Engine.cpp index c2ac897c36..22842564e1 100644 --- a/source/adios2/engine/bp5/BP5Engine.cpp +++ b/source/adios2/engine/bp5/BP5Engine.cpp @@ -193,7 +193,7 @@ void BP5Engine::ParseParams(IO &io, struct BP5Params &Params) parameter = helper::StringToByteUnits( value, "for Parameter key=" + key + "in call to Open"); parameter = - helper::StringTo(value, " in Parameter key=" + key); + helper::StringTo(value, " in Parameter key=" + key); } }; @@ -263,6 +263,33 @@ void BP5Engine::ParseParams(IO &io, struct BP5Params &Params) } }; + auto lf_SetAggregationTypeParameter = [&](const std::string key, + int ¶meter, int def) { + auto itKey = io.m_Parameters.find(key); + parameter = def; + if (itKey != io.m_Parameters.end()) + { + std::string value = itKey->second; + std::transform(value.begin(), value.end(), value.begin(), + ::tolower); + if (value == "everyonewrites" || value == "auto") + { + parameter = (int)AggregationType::EveryoneWrites; + } + else if (value == "twolevelshm") + { + parameter = (int)AggregationType::TwoLevelShm; + } + else + { + throw std::invalid_argument( + "ERROR: Unknown BP5 AggregationType parameter \"" + value + + "\" (must be \"auto\", \"everyonewrites\" or " + "\"twolevelshm\""); + } + } + }; + #define get_params(Param, Type, Typedecl, Default) \ lf_Set##Type##Parameter(#Param, Params.Param, Default); BP5_FOREACH_PARAMETER_TYPE_4ARGS(get_params); diff --git a/source/adios2/engine/bp5/BP5Engine.h b/source/adios2/engine/bp5/BP5Engine.h index 609197f5c4..b38a698075 100644 --- a/source/adios2/engine/bp5/BP5Engine.h +++ b/source/adios2/engine/bp5/BP5Engine.h @@ -93,6 +93,12 @@ class BP5Engine BufferVType UseBufferV = BufferVType::ChunkVType; + enum class AggregationType + { + EveryoneWrites, + TwoLevelShm + }; + #define BP5_FOREACH_PARAMETER_TYPE_4ARGS(MACRO) \ MACRO(OpenTimeoutSecs, Int, int, 3600) \ MACRO(BeginStepPollingFrequencySecs, Int, int, 0) \ @@ -103,6 +109,9 @@ class BP5Engine MACRO(verbose, Int, int, 0) \ MACRO(CollectiveMetadata, Bool, bool, true) \ MACRO(NumAggregators, UInt, unsigned int, 999999) \ + MACRO(NumSubFiles, UInt, unsigned int, 999999) \ + MACRO(AggregationType, AggregationType, int, \ + (int)AggregationType::EveryoneWrites) \ MACRO(AsyncTasks, Bool, bool, true) \ MACRO(GrowthFactor, Float, float, DefaultBufferGrowthFactor) \ MACRO(InitialBufferSize, SizeBytes, size_t, DefaultInitialBufferSize) \ diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index 0f77952d24..8628ba93c1 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -131,10 +131,29 @@ uint64_t BP5Writer::WriteMetadata( return MetaDataSize; } -static const uint64_t PAGE_SIZE = 65536; // 64KB void BP5Writer::WriteData(format::BufferV *Data) { format::BufferV::BufferV_iovec DataVec = Data->DataVec(); + if (m_Parameters.AggregationType == (int)AggregationType::EveryoneWrites) + { + WriteData_EveryoneWrites(DataVec); + } + else if (m_Parameters.AggregationType == (int)AggregationType::TwoLevelShm) + { + WriteData_TwoLevelShm(DataVec); + } + else + { + throw std::invalid_argument( + "Aggregation method " + + std::to_string(m_Parameters.AggregationType) + + "is not supported in BP5"); + } +} + +void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec) +{ + constexpr uint64_t PAGE_SIZE = 65536; // 64KB // new step writing starts at offset m_DataPos on aggregator // others will wait for the position to arrive from the rank below @@ -338,15 +357,24 @@ void BP5Writer::Init() m_BP5Serializer.m_Engine = this; m_RankMPI = m_Comm.Rank(); InitParameters(); - if (m_Parameters.NumAggregators > static_cast(m_Comm.Size())) - { - m_Parameters.NumAggregators = static_cast(m_Comm.Size()); - } + // in BP5, aggregation is "always on", but processes may be alone, so // m_Aggregator.m_IsActive is always true // m_Aggregator.m_Comm.Rank() will always succeed (not abort) // m_Aggregator.m_SubFileIndex is always set - m_Aggregator.Init(m_Parameters.NumAggregators, m_Comm); + m_Aggregator.Init(m_Parameters.NumAggregators, m_Parameters.NumSubFiles, + m_Comm); + + std::cout << "Rank " << m_RankMPI << " aggr? " + << m_Aggregator.m_IsAggregator << " master? " + << m_Aggregator.m_IsMasterAggregator + << " aggr size = " << m_Aggregator.m_Size + << " rank = " << m_Aggregator.m_Rank + << " subfile = " << m_Aggregator.m_SubStreamIndex + << " type = " << m_Parameters.AggregationType + + << std::endl; + InitTransports(); InitBPBuffer(); } @@ -369,6 +397,21 @@ void BP5Writer::InitParameters() ParseParams(m_IO, m_Parameters); m_WriteToBB = !(m_Parameters.BurstBufferPath.empty()); m_DrainBB = m_WriteToBB && m_Parameters.BurstBufferDrain; + + size_t numNodes = m_Aggregator.PreInit(m_Comm); + if (m_Parameters.NumAggregators > static_cast(m_Comm.Size())) + { + m_Parameters.NumAggregators = static_cast(m_Comm.Size()); + } + + if (m_Parameters.NumSubFiles > m_Parameters.NumAggregators) + { + m_Parameters.NumSubFiles = m_Parameters.NumAggregators; + } + if (m_Parameters.AggregationType == (int)AggregationType::EveryoneWrites) + { + m_Parameters.NumSubFiles = m_Parameters.NumAggregators; + } } void BP5Writer::InitTransports() diff --git a/source/adios2/engine/bp5/BP5Writer.h b/source/adios2/engine/bp5/BP5Writer.h index cef94b1ba2..69bfab7390 100644 --- a/source/adios2/engine/bp5/BP5Writer.h +++ b/source/adios2/engine/bp5/BP5Writer.h @@ -13,7 +13,7 @@ #include "adios2/core/Engine.h" #include "adios2/engine/bp5/BP5Engine.h" #include "adios2/helper/adiosComm.h" -#include "adios2/toolkit/aggregator/mpi/MPIChain.h" +#include "adios2/toolkit/aggregator/mpi/MPIShmChain.h" #include "adios2/toolkit/burstbuffer/FileDrainerSingleThread.h" #include "adios2/toolkit/format/bp5/BP5Serializer.h" #include "adios2/toolkit/transportman/TransportMan.h" @@ -127,6 +127,8 @@ class BP5Writer : public BP5Engine, public core::Engine /** Write Data to disk, in an aggregator chain */ void WriteData(format::BufferV *Data); + void WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec); + void WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec); void PopulateMetadataIndexFileContent( format::BufferSTL &buffer, const uint64_t currentStep, @@ -140,17 +142,8 @@ class BP5Writer : public BP5Engine, public core::Engine void MarshalAttributes(); - /** - * N-to-N data buffers writes, including metadata file - * @param transportIndex - */ - // void WriteData(const bool isFinal, const int transportIndex = -1); - - /** - * N-to-M (aggregation) data buffers writes, including metadata file - * @param transportIndex - */ - void AggregateWriteData(const bool isFinal, const int transportIndex = -1); + /* Shmem aggregator functions */ + void WriteMyOwnData(format::BufferV::BufferV_iovec DataVec); template T *BufferDataCommon(const size_t payloadOffset, @@ -160,7 +153,7 @@ class BP5Writer : public BP5Engine, public core::Engine void PerformPutCommon(Variable &variable); /** manages all communication tasks in aggregation */ - aggregator::MPIChain m_Aggregator; + aggregator::MPIShmChain m_Aggregator; private: // updated during WriteMetaData diff --git a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp new file mode 100644 index 0000000000..ccfa243a28 --- /dev/null +++ b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp @@ -0,0 +1,128 @@ +/* + * Distributed under the OSI-approved Apache License, Version 2.0. See + * accompanying file Copyright.txt for details. + * + * BP5Writer.cpp + * + */ + +#include "BP5Writer.h" + +#include "adios2/common/ADIOSMacros.h" +#include "adios2/core/IO.h" +#include "adios2/helper/adiosFunctions.h" //CheckIndexRange +#include "adios2/toolkit/format/buffer/chunk/ChunkV.h" +#include "adios2/toolkit/format/buffer/malloc/MallocV.h" +#include "adios2/toolkit/transport/file/FileFStream.h" +#include + +#include +#include + +namespace adios2 +{ +namespace core +{ +namespace engine +{ + +using namespace adios2::format; + +void BP5Writer::WriteMyOwnData(format::BufferV::BufferV_iovec DataVec) +{ + m_StartDataPos = m_DataPos; + int i = 0; + while (DataVec[i].iov_base != NULL) + { + if (i == 0) + { + m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, + DataVec[i].iov_len, m_StartDataPos); + } + else + { + m_FileDataManager.WriteFiles((char *)DataVec[i].iov_base, + DataVec[i].iov_len); + } + m_DataPos += DataVec[i].iov_len; + i++; + } +} + +constexpr uint64_t PAGE_SIZE = 65536; // 64KB +void BP5Writer::WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec) +{ + // new step writing starts at offset m_DataPos on aggregator + // others will wait for the position to arrive from the rank below + + if (m_Aggregator.m_Comm.Rank() > 0) + { + m_Aggregator.m_Comm.Recv(&m_DataPos, 1, m_Aggregator.m_Comm.Rank() - 1, + 0, "Chain token in BP5Writer::WriteData"); + } + m_StartDataPos = m_DataPos; + + if (m_Aggregator.m_Comm.Rank() < m_Aggregator.m_Comm.Size() - 1) + { + int i = 0; + uint64_t nextWriterPos = m_DataPos; + while (DataVec[i].iov_base != NULL) + { + nextWriterPos += DataVec[i].iov_len; + i++; + } + // align to PAGE_SIZE + nextWriterPos += PAGE_SIZE - (nextWriterPos % PAGE_SIZE); + m_Aggregator.m_Comm.Isend(&nextWriterPos, 1, + m_Aggregator.m_Comm.Rank() + 1, 0, + "Chain token in BP5Writer::WriteData"); + } + + /* Aggregator starts with writing its own data */ + if (m_Aggregator.m_Comm.Rank() == 0) + { + WriteMyOwnData(DataVec); + } + + int i = 0; + while (DataVec[i].iov_base != NULL) + { + if (i == 0) + { + m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, + DataVec[i].iov_len, m_StartDataPos); + } + else + { + m_FileDataManager.WriteFiles((char *)DataVec[i].iov_base, + DataVec[i].iov_len); + } + m_DataPos += DataVec[i].iov_len; + i++; + } + + if (m_Aggregator.m_Comm.Size() > 1) + { + // at the end, last rank sends back the final data pos to first rank + // so it can update its data pos + if (m_Aggregator.m_Comm.Rank() == m_Aggregator.m_Comm.Size() - 1) + { + // align to PAGE_SIZE + m_DataPos += PAGE_SIZE - (m_DataPos % PAGE_SIZE); + m_Aggregator.m_Comm.Isend( + &m_DataPos, 1, 0, 0, + "Final chain token in BP5Writer::WriteData"); + } + if (m_Aggregator.m_Comm.Rank() == 0) + { + m_Aggregator.m_Comm.Recv(&m_DataPos, 1, + m_Aggregator.m_Comm.Size() - 1, 0, + "Chain token in BP5Writer::WriteData"); + } + } + delete[] DataVec; +} + +} // end namespace engine +} // end namespace core +} // end namespace adios2 diff --git a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp index f995ba27e6..76604ad5f0 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp @@ -28,9 +28,12 @@ MPIAggregator::~MPIAggregator() } } -void MPIAggregator::Init(const size_t subStreams, +void MPIAggregator::Init(const size_t numAggregators, const size_t subStreams, helper::Comm const &parentComm) + { + m_NumAggregators = numAggregators; + m_SubStreams = subStreams; } void MPIAggregator::SwapBuffers(const int step) noexcept {} diff --git a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h index ac61425373..201181e665 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h @@ -31,6 +31,11 @@ class MPIAggregator /** current substream index from 0 to m_SubStreams-1 */ size_t m_SubStreamIndex = 0; + /** total number of aggregators + * (BP3/BP4 uses aggregators = substreams) + */ + size_t m_NumAggregators = 0; + /** split Communicator for a substream: producers and consumer (rank=0) */ helper::Comm m_Comm; @@ -57,7 +62,8 @@ class MPIAggregator virtual ~MPIAggregator(); - virtual void Init(const size_t subStreams, helper::Comm const &parentComm); + virtual void Init(const size_t numAggregators, const size_t subStreams, + helper::Comm const &parentComm); struct ExchangeRequests { diff --git a/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp index eea9c50f15..1bd1e298fa 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp @@ -18,8 +18,11 @@ namespace aggregator MPIChain::MPIChain() : MPIAggregator() {} -void MPIChain::Init(const size_t subStreams, helper::Comm const &parentComm) +void MPIChain::Init(const size_t numAggregators, const size_t subStreams, + helper::Comm const &parentComm) { + /* numAggregators ignored here as BP3/BP4 uses substreams = aggregators */ + m_NumAggregators = subStreams; if (subStreams > 0) { InitComm(subStreams, parentComm); diff --git a/source/adios2/toolkit/aggregator/mpi/MPIChain.h b/source/adios2/toolkit/aggregator/mpi/MPIChain.h index a26895d8d2..6cc1f8a288 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIChain.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIChain.h @@ -22,11 +22,13 @@ class MPIChain : public MPIAggregator { public: + /* Chain aggregator used by BP3/BP4 */ MPIChain(); ~MPIChain() = default; - void Init(const size_t subStreams, helper::Comm const &parentComm) final; + void Init(const size_t numAggregators, const size_t subStreams, + helper::Comm const &parentComm) final; ExchangeRequests IExchange(format::Buffer &buffer, const int step) final; diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp new file mode 100644 index 0000000000..f85b7eade3 --- /dev/null +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp @@ -0,0 +1,208 @@ +/* + * Distributed under the OSI-approved Apache License, Version 2.0. See + * accompanying file Copyright.txt for details. + * + * MPIShmChain.h + * + * Created on: July 5, 2021 + * Author: Norbert Podhorszki pnorbert@ornl.gov + */ +#include "MPIShmChain.h" + +namespace adios2 +{ +namespace aggregator +{ + +MPIShmChain::MPIShmChain() : MPIAggregator() {} + +MPIShmChain::~MPIShmChain() +{ + if (m_IsActive) + { + m_NodeComm.Free("free per-node comm in ~MPIShmChain()"); + m_OnePerNodeComm.Free("free chain of nodes in ~MPIShmChain()"); + m_AllAggregatorsComm.Free( + "free comm of all aggregators in ~MPIShmChain()"); + m_AggregatorChainComm.Free( + "free chains of aggregators in ~MPIShmChain()"); + } +} + +size_t MPIShmChain::PreInit(helper::Comm const &parentComm) +{ + /* Communicator connecting ranks on each Compute Node */ + m_NodeComm = parentComm.GroupByShm("creating per-node comm at Open"); + int NodeRank = m_NodeComm.Rank(); + + /* + * Communicators connecting rank N of each node + * We are only interested in the chain of rank 0s + */ + int color = (NodeRank ? 1 : 0); + m_OnePerNodeComm = + parentComm.Split(color, 0, "creating chain of nodes at Open"); + + /* Number of nodes */ + if (!NodeRank) + { + m_NumNodes = static_cast(m_OnePerNodeComm.Size()); + } + m_NumNodes = m_NodeComm.BroadcastValue(m_NumNodes, 0); + PreInitCalled = true; + return m_NumNodes; +} + +void MPIShmChain::Init(const size_t numAggregators, const size_t subStreams, + helper::Comm const &parentComm) +{ + if (!PreInitCalled) + { + PreInit(parentComm); + } + + // int AllRank = parentComm.Rank(); + // int AllSize = parentComm.Size(); + int NodeRank = m_NodeComm.Rank(); + size_t NodeSize = static_cast(m_NodeComm.Size()); + + /* Number of aggregators per node */ + size_t aggregatorPerNode = numAggregators / m_NumNodes; + if (aggregatorPerNode == 0) + { + aggregatorPerNode = 1; + } + if (aggregatorPerNode > NodeSize) + { + aggregatorPerNode = NodeSize; + } + + /* Create main communicator that splits the node comm into one or more + * aggregator chains */ + float k = + static_cast(NodeSize) / static_cast(aggregatorPerNode); + float c = static_cast(NodeRank) / k; + int color = static_cast(c); + m_Comm = m_NodeComm.Split(color, 0, "creating aggregator groups at Open"); + m_Rank = m_Comm.Rank(); + m_Size = m_Comm.Size(); + if (m_Rank != 0) + { + m_IsAggregator = false; + m_IsMasterAggregator = false; + } + + /* Identify parent rank of aggregator process within each chain */ + if (!m_Rank) + { + m_AggregatorRank = parentComm.Rank(); + } + m_AggregatorRank = m_Comm.BroadcastValue(m_AggregatorRank, 0); + + /* Communicator for all Aggregators */ + color = (m_Rank ? 1 : 0); + m_AllAggregatorsComm = + parentComm.Split(color, 0, "creating comm of all aggregators at Open"); + + /* Total number of aggregators */ + if (!NodeRank) + { + m_NumAggregators = static_cast(m_AllAggregatorsComm.Size()); + } + m_NumAggregators = m_NodeComm.BroadcastValue(m_NumAggregators); + + /* Number of substreams */ + m_SubStreams = subStreams; + if (m_SubStreams == 0) + { + m_SubStreams = 1; + } + if (m_SubStreams > m_NumAggregators) + { + m_SubStreams = m_NumAggregators; + } + + if (!m_Rank) + { + k = static_cast(m_NumAggregators) / + static_cast(m_SubStreams); + /* 1.0 <= k <= m_NumAggregators */ + c = static_cast(m_AllAggregatorsComm.Rank()) / k; + m_SubStreamIndex = static_cast(c); + } + m_SubStreamIndex = m_Comm.BroadcastValue(m_SubStreamIndex); + + /* Create the communicator to connect aggregators writing to the same + * substream */ + color = m_SubStreamIndex; + m_AggregatorChainComm = m_AllAggregatorsComm.Split( + color, 0, "creating chains of aggregators at Open"); + + if (m_AggregatorChainComm.Rank() != 0) + { + m_IsMasterAggregator = false; + } + + m_IsActive = true; + + HandshakeLinks(); +} + +// PRIVATE +void MPIShmChain::HandshakeLinks() +{ + int link = -1; + + helper::Comm::Req sendRequest; + if (m_Rank > 0) // send + { + sendRequest = m_Comm.Isend( + &m_Rank, 1, m_Rank - 1, 0, + "Isend handshake with neighbor, MPIChain aggregator, at Open"); + } + + if (m_Rank < m_Size - 1) // receive + { + helper::Comm::Req receiveRequest = m_Comm.Irecv( + &link, 1, m_Rank + 1, 0, + "Irecv handshake with neighbor, MPIChain aggregator, at Open"); + + receiveRequest.Wait("Irecv Wait handshake with neighbor, MPIChain " + "aggregator, at Open"); + } + + if (m_Rank > 0) + { + sendRequest.Wait("Isend wait handshake with neighbor, MPIChain " + "aggregator, at Open"); + } +} + +/*********************** + * Remove these + ***********************/ + +MPIShmChain::ExchangeRequests MPIShmChain::IExchange(format::Buffer &buffer, + const int step) +{ + ExchangeRequests requests; + return requests; +} + +MPIShmChain::ExchangeAbsolutePositionRequests +MPIShmChain::IExchangeAbsolutePosition(format::Buffer &buffer, const int step) +{ + ExchangeAbsolutePositionRequests requests; + return requests; +} + +void MPIShmChain::Wait(ExchangeRequests &requests, const int step) { return; } + +void MPIShmChain::WaitAbsolutePosition( + ExchangeAbsolutePositionRequests &requests, const int step) +{ + return; +} + +} // end namespace aggregator +} // end namespace adios2 diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h new file mode 100644 index 0000000000..04139e6325 --- /dev/null +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h @@ -0,0 +1,104 @@ +/* + * Distributed under the OSI-approved Apache License, Version 2.0. See + * accompanying file Copyright.txt for details. + * + * MPIShmChain.h + * + * Created on: July 5, 2021 + * Author: Norbert Podhorszki pnorbert@ornl.gov + * + */ + +#ifndef ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPICSHMHAIN_H_ +#define ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPISHMCHAIN_H_ + +#include "adios2/toolkit/aggregator/mpi/MPIAggregator.h" + +namespace adios2 +{ +namespace aggregator +{ + +/** A one- or two-layer aggregator chain for using Shared memory within a + * compute node. + * Use MPI split type to group processes within a node into one chain. + * Depending on the number of aggregators, multiple nodes may be merged into + * a two-layer chain, (multiple Aggregators and one Master). + * Otherwise its a simple chain with one Aggregator=Master. + * + * m_Comm is the communicator that split for each node + * + */ + +class MPIShmChain : public MPIAggregator +{ + +public: + MPIShmChain(); + + ~MPIShmChain(); + + /* Create a per-node communicator and return number of nodes */ + size_t PreInit(helper::Comm const &parentComm); + + void Init(const size_t numAggregators, const size_t subStreams, + helper::Comm const &parentComm) final; + + /** + * true: the Master (aggregator) process in the chain + * always m_Rank == m_Comm.Rank() == 0 for a master aggregator + * same as (m_AggregatorChainComm.Rank() == 0) + */ + bool m_IsMasterAggregator = true; + + /* These are not used and must be removed */ + ExchangeRequests IExchange(format::Buffer &buffer, const int step) final; + + ExchangeAbsolutePositionRequests + IExchangeAbsolutePosition(format::Buffer &buffer, const int step) final; + + void Wait(ExchangeRequests &requests, const int step) final; + + void WaitAbsolutePosition(ExchangeAbsolutePositionRequests &requests, + const int step) final; + +private: + void HandshakeLinks(); + + /* + Variables set in PreInit + */ + + bool PreInitCalled = false; + /* Communicator per compute node */ + helper::Comm m_NodeComm; + /* Communicator connecting rank 0 on each node + Useful only on rank 0s of m_NodeComm */ + helper::Comm m_OnePerNodeComm; + /* Number of Compute Nodes + * (size of m_OnePerNodeComm created from rank 0s of m_NodeComm) + */ + size_t m_NumNodes; + + /* + Variables set in Init + */ + + /* Communicator connecting all aggregators + (rank 0 of each aggregator group) + Useful only on aggregators themselves + */ + helper::Comm m_AllAggregatorsComm; + + /* Communicator connecting the aggregators + that write to the same substream. + rank 0 becomes a MasterAggregator + Useful only on aggregators themselves + */ + helper::Comm m_AggregatorChainComm; +}; + +} // end namespace aggregator +} // end namespace adios2 + +#endif /* ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPISHMCHAIN_H_ */ From 87faca2a2c5754e1e15f0dc05b346e66e36e6a6b Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Mon, 12 Jul 2021 10:11:16 -0400 Subject: [PATCH 04/18] EveryoneWrites, EveryoneWritesSerial, TwoLevelShm aggregation types --- source/adios2/engine/bp5/BP5Engine.cpp | 4 +++ source/adios2/engine/bp5/BP5Engine.h | 4 ++- source/adios2/engine/bp5/BP5Writer.cpp | 43 +++++++++++++++++++------- source/adios2/engine/bp5/BP5Writer.h | 3 +- 4 files changed, 40 insertions(+), 14 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Engine.cpp b/source/adios2/engine/bp5/BP5Engine.cpp index 22842564e1..07ae493273 100644 --- a/source/adios2/engine/bp5/BP5Engine.cpp +++ b/source/adios2/engine/bp5/BP5Engine.cpp @@ -276,6 +276,10 @@ void BP5Engine::ParseParams(IO &io, struct BP5Params &Params) { parameter = (int)AggregationType::EveryoneWrites; } + else if (value == "everyonewritesserial") + { + parameter = (int)AggregationType::EveryoneWritesSerial; + } else if (value == "twolevelshm") { parameter = (int)AggregationType::TwoLevelShm; diff --git a/source/adios2/engine/bp5/BP5Engine.h b/source/adios2/engine/bp5/BP5Engine.h index b38a698075..5415b4a4cc 100644 --- a/source/adios2/engine/bp5/BP5Engine.h +++ b/source/adios2/engine/bp5/BP5Engine.h @@ -96,7 +96,9 @@ class BP5Engine enum class AggregationType { EveryoneWrites, - TwoLevelShm + EveryoneWritesSerial, + TwoLevelShm, + Auto }; #define BP5_FOREACH_PARAMETER_TYPE_4ARGS(MACRO) \ diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index 8628ba93c1..e5f32d7b3a 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -134,16 +134,18 @@ uint64_t BP5Writer::WriteMetadata( void BP5Writer::WriteData(format::BufferV *Data) { format::BufferV::BufferV_iovec DataVec = Data->DataVec(); - if (m_Parameters.AggregationType == (int)AggregationType::EveryoneWrites) - { - WriteData_EveryoneWrites(DataVec); - } - else if (m_Parameters.AggregationType == (int)AggregationType::TwoLevelShm) - { + switch (m_Parameters.AggregationType) + { + case (int)AggregationType::EveryoneWrites: + WriteData_EveryoneWrites(DataVec, false); + break; + case (int)AggregationType::EveryoneWritesSerial: + WriteData_EveryoneWrites(DataVec, true); + break; + case (int)AggregationType::TwoLevelShm: WriteData_TwoLevelShm(DataVec); - } - else - { + break; + default: throw std::invalid_argument( "Aggregation method " + std::to_string(m_Parameters.AggregationType) + @@ -151,7 +153,8 @@ void BP5Writer::WriteData(format::BufferV *Data) } } -void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec) +void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, + bool SerializedWriters) { constexpr uint64_t PAGE_SIZE = 65536; // 64KB @@ -165,8 +168,10 @@ void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec) } m_StartDataPos = m_DataPos; - if (m_Aggregator.m_Comm.Rank() < m_Aggregator.m_Comm.Size() - 1) + if (!SerializedWriters && + m_Aggregator.m_Comm.Rank() < m_Aggregator.m_Comm.Size() - 1) { + /* Send the token before writing so everyone can start writing asap */ int i = 0; uint64_t nextWriterPos = m_DataPos; while (DataVec[i].iov_base != NULL) @@ -198,6 +203,18 @@ void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec) i++; } + if (SerializedWriters && + m_Aggregator.m_Comm.Rank() < m_Aggregator.m_Comm.Size() - 1) + { + /* send token now, effectively serializing the writers in the chain */ + uint64_t nextWriterPos = m_DataPos; + // align to PAGE_SIZE + nextWriterPos += PAGE_SIZE - (nextWriterPos % PAGE_SIZE); + m_Aggregator.m_Comm.Isend(&nextWriterPos, 1, + m_Aggregator.m_Comm.Rank() + 1, 0, + "Chain token in BP5Writer::WriteData"); + } + if (m_Aggregator.m_Comm.Size() > 1) { // at the end, last rank sends back the final data pos to first rank @@ -408,7 +425,9 @@ void BP5Writer::InitParameters() { m_Parameters.NumSubFiles = m_Parameters.NumAggregators; } - if (m_Parameters.AggregationType == (int)AggregationType::EveryoneWrites) + if (m_Parameters.AggregationType == (int)AggregationType::EveryoneWrites || + m_Parameters.AggregationType == + (int)AggregationType::EveryoneWritesSerial) { m_Parameters.NumSubFiles = m_Parameters.NumAggregators; } diff --git a/source/adios2/engine/bp5/BP5Writer.h b/source/adios2/engine/bp5/BP5Writer.h index 69bfab7390..66e9720682 100644 --- a/source/adios2/engine/bp5/BP5Writer.h +++ b/source/adios2/engine/bp5/BP5Writer.h @@ -127,7 +127,8 @@ class BP5Writer : public BP5Engine, public core::Engine /** Write Data to disk, in an aggregator chain */ void WriteData(format::BufferV *Data); - void WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec); + void WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, + bool SerializedWriters); void WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec); void PopulateMetadataIndexFileContent( From b026ffcf609addf2b6457b02d50cd57ef805382a Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Tue, 13 Jul 2021 08:24:46 -0400 Subject: [PATCH 05/18] rework aggregator class to make MPIAggregator a non-abstract class so that it can be used as a base class pointer to derived objects in BP5Writer --- source/adios2/engine/bp3/BP3Writer.cpp | 4 +- source/adios2/engine/bp4/BP4Writer.cpp | 4 +- source/adios2/engine/bp5/BP5Engine.h | 1 + source/adios2/engine/bp5/BP5Writer.cpp | 143 +++++++++++------- source/adios2/engine/bp5/BP5Writer.h | 11 +- .../engine/bp5/BP5Writer_TwoLevelShm.cpp | 42 +++-- .../toolkit/aggregator/mpi/MPIAggregator.cpp | 18 +-- .../toolkit/aggregator/mpi/MPIAggregator.h | 33 +--- .../toolkit/aggregator/mpi/MPIChain.cpp | 2 + .../adios2/toolkit/aggregator/mpi/MPIChain.h | 31 +++- .../toolkit/aggregator/mpi/MPIShmChain.cpp | 41 ++--- .../toolkit/aggregator/mpi/MPIShmChain.h | 13 +- 12 files changed, 164 insertions(+), 179 deletions(-) diff --git a/source/adios2/engine/bp3/BP3Writer.cpp b/source/adios2/engine/bp3/BP3Writer.cpp index 4cd48cdfdb..5e853716ba 100644 --- a/source/adios2/engine/bp3/BP3Writer.cpp +++ b/source/adios2/engine/bp3/BP3Writer.cpp @@ -389,10 +389,10 @@ void BP3Writer::AggregateWriteData(const bool isFinal, const int transportIndex) // async? for (int r = 0; r < m_BP3Serializer.m_Aggregator.m_Size; ++r) { - aggregator::MPIAggregator::ExchangeRequests dataRequests = + aggregator::MPIChain::ExchangeRequests dataRequests = m_BP3Serializer.m_Aggregator.IExchange(m_BP3Serializer.m_Data, r); - aggregator::MPIAggregator::ExchangeAbsolutePositionRequests + aggregator::MPIChain::ExchangeAbsolutePositionRequests absolutePositionRequests = m_BP3Serializer.m_Aggregator.IExchangeAbsolutePosition( m_BP3Serializer.m_Data, r); diff --git a/source/adios2/engine/bp4/BP4Writer.cpp b/source/adios2/engine/bp4/BP4Writer.cpp index 1e4a2b016c..062318acad 100644 --- a/source/adios2/engine/bp4/BP4Writer.cpp +++ b/source/adios2/engine/bp4/BP4Writer.cpp @@ -746,10 +746,10 @@ void BP4Writer::AggregateWriteData(const bool isFinal, const int transportIndex) // async? for (int r = 0; r < m_BP4Serializer.m_Aggregator.m_Size; ++r) { - aggregator::MPIAggregator::ExchangeRequests dataRequests = + aggregator::MPIChain::ExchangeRequests dataRequests = m_BP4Serializer.m_Aggregator.IExchange(m_BP4Serializer.m_Data, r); - aggregator::MPIAggregator::ExchangeAbsolutePositionRequests + aggregator::MPIChain::ExchangeAbsolutePositionRequests absolutePositionRequests = m_BP4Serializer.m_Aggregator.IExchangeAbsolutePosition( m_BP4Serializer.m_Data, r); diff --git a/source/adios2/engine/bp5/BP5Engine.h b/source/adios2/engine/bp5/BP5Engine.h index 5415b4a4cc..256beb5eab 100644 --- a/source/adios2/engine/bp5/BP5Engine.h +++ b/source/adios2/engine/bp5/BP5Engine.h @@ -112,6 +112,7 @@ class BP5Engine MACRO(CollectiveMetadata, Bool, bool, true) \ MACRO(NumAggregators, UInt, unsigned int, 999999) \ MACRO(NumSubFiles, UInt, unsigned int, 999999) \ + MACRO(FileSystemPageSize, UInt, unsigned int, 65536) \ MACRO(AggregationType, AggregationType, int, \ (int)AggregationType::EveryoneWrites) \ MACRO(AsyncTasks, Bool, bool, true) \ diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index e5f32d7b3a..48ceddeb7b 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -156,20 +156,24 @@ void BP5Writer::WriteData(format::BufferV *Data) void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, bool SerializedWriters) { - constexpr uint64_t PAGE_SIZE = 65536; // 64KB + const aggregator::MPIChain *a = + dynamic_cast(m_Aggregator); // new step writing starts at offset m_DataPos on aggregator // others will wait for the position to arrive from the rank below - if (m_Aggregator.m_Comm.Rank() > 0) + if (a->m_Comm.Rank() > 0) { - m_Aggregator.m_Comm.Recv(&m_DataPos, 1, m_Aggregator.m_Comm.Rank() - 1, - 0, "Chain token in BP5Writer::WriteData"); + a->m_Comm.Recv(&m_DataPos, 1, a->m_Comm.Rank() - 1, 0, + "Chain token in BP5Writer::WriteData"); } + + // align to PAGE_SIZE + m_DataPos += m_Parameters.FileSystemPageSize - + (m_DataPos % m_Parameters.FileSystemPageSize); m_StartDataPos = m_DataPos; - if (!SerializedWriters && - m_Aggregator.m_Comm.Rank() < m_Aggregator.m_Comm.Size() - 1) + if (!SerializedWriters && a->m_Comm.Rank() < a->m_Comm.Size() - 1) { /* Send the token before writing so everyone can start writing asap */ int i = 0; @@ -179,11 +183,8 @@ void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, nextWriterPos += DataVec[i].iov_len; i++; } - // align to PAGE_SIZE - nextWriterPos += PAGE_SIZE - (nextWriterPos % PAGE_SIZE); - m_Aggregator.m_Comm.Isend(&nextWriterPos, 1, - m_Aggregator.m_Comm.Rank() + 1, 0, - "Chain token in BP5Writer::WriteData"); + a->m_Comm.Isend(&nextWriterPos, 1, a->m_Comm.Rank() + 1, 0, + "Chain token in BP5Writer::WriteData"); } int i = 0; @@ -191,6 +192,7 @@ void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, { if (i == 0) { + m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, DataVec[i].iov_len, m_StartDataPos); } @@ -203,37 +205,30 @@ void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, i++; } - if (SerializedWriters && - m_Aggregator.m_Comm.Rank() < m_Aggregator.m_Comm.Size() - 1) + if (SerializedWriters && a->m_Comm.Rank() < a->m_Comm.Size() - 1) { /* send token now, effectively serializing the writers in the chain */ uint64_t nextWriterPos = m_DataPos; - // align to PAGE_SIZE - nextWriterPos += PAGE_SIZE - (nextWriterPos % PAGE_SIZE); - m_Aggregator.m_Comm.Isend(&nextWriterPos, 1, - m_Aggregator.m_Comm.Rank() + 1, 0, - "Chain token in BP5Writer::WriteData"); + a->m_Comm.Isend(&nextWriterPos, 1, a->m_Comm.Rank() + 1, 0, + "Chain token in BP5Writer::WriteData"); } - if (m_Aggregator.m_Comm.Size() > 1) + if (a->m_Comm.Size() > 1) { // at the end, last rank sends back the final data pos to first rank // so it can update its data pos - if (m_Aggregator.m_Comm.Rank() == m_Aggregator.m_Comm.Size() - 1) + if (a->m_Comm.Rank() == a->m_Comm.Size() - 1) { - // align to PAGE_SIZE - m_DataPos += PAGE_SIZE - (m_DataPos % PAGE_SIZE); - m_Aggregator.m_Comm.Isend( - &m_DataPos, 1, 0, 0, - "Final chain token in BP5Writer::WriteData"); + a->m_Comm.Isend(&m_DataPos, 1, 0, 0, + "Final chain token in BP5Writer::WriteData"); } - if (m_Aggregator.m_Comm.Rank() == 0) + if (a->m_Comm.Rank() == 0) { - m_Aggregator.m_Comm.Recv(&m_DataPos, 1, - m_Aggregator.m_Comm.Size() - 1, 0, - "Chain token in BP5Writer::WriteData"); + a->m_Comm.Recv(&m_DataPos, 1, a->m_Comm.Size() - 1, 0, + "Chain token in BP5Writer::WriteData"); } } + delete[] DataVec; } @@ -374,24 +369,7 @@ void BP5Writer::Init() m_BP5Serializer.m_Engine = this; m_RankMPI = m_Comm.Rank(); InitParameters(); - - // in BP5, aggregation is "always on", but processes may be alone, so - // m_Aggregator.m_IsActive is always true - // m_Aggregator.m_Comm.Rank() will always succeed (not abort) - // m_Aggregator.m_SubFileIndex is always set - m_Aggregator.Init(m_Parameters.NumAggregators, m_Parameters.NumSubFiles, - m_Comm); - - std::cout << "Rank " << m_RankMPI << " aggr? " - << m_Aggregator.m_IsAggregator << " master? " - << m_Aggregator.m_IsMasterAggregator - << " aggr size = " << m_Aggregator.m_Size - << " rank = " << m_Aggregator.m_Rank - << " subfile = " << m_Aggregator.m_SubStreamIndex - << " type = " << m_Parameters.AggregationType - - << std::endl; - + InitAggregator(); InitTransports(); InitBPBuffer(); } @@ -415,7 +393,6 @@ void BP5Writer::InitParameters() m_WriteToBB = !(m_Parameters.BurstBufferPath.empty()); m_DrainBB = m_WriteToBB && m_Parameters.BurstBufferDrain; - size_t numNodes = m_Aggregator.PreInit(m_Comm); if (m_Parameters.NumAggregators > static_cast(m_Comm.Size())) { m_Parameters.NumAggregators = static_cast(m_Comm.Size()); @@ -425,11 +402,49 @@ void BP5Writer::InitParameters() { m_Parameters.NumSubFiles = m_Parameters.NumAggregators; } +} + +void BP5Writer::InitAggregator() +{ + // in BP5, aggregation is "always on", but processes may be alone, so + // m_Aggregator.m_IsActive is always true + // m_Aggregator.m_Comm.Rank() will always succeed (not abort) + // m_Aggregator.m_SubFileIndex is always set if (m_Parameters.AggregationType == (int)AggregationType::EveryoneWrites || m_Parameters.AggregationType == (int)AggregationType::EveryoneWritesSerial) { m_Parameters.NumSubFiles = m_Parameters.NumAggregators; + m_AggregatorEveroneWrites.Init(m_Parameters.NumAggregators, + m_Parameters.NumSubFiles, m_Comm); + m_IAmDraining = m_AggregatorEveroneWrites.m_IsAggregator; + m_IAmWritingDataHeader = m_AggregatorEveroneWrites.m_IsAggregator; + m_EveryoneWrites = true; + m_IAmWritingData = true; + m_Aggregator = static_cast( + &m_AggregatorEveroneWrites); + } + else + { + size_t numNodes = m_AggregatorTwoLevelShm.PreInit(m_Comm); + m_AggregatorTwoLevelShm.Init(m_Parameters.NumAggregators, + m_Parameters.NumSubFiles, m_Comm); + + std::cout << "Rank " << m_RankMPI << " aggr? " + << m_AggregatorTwoLevelShm.m_IsAggregator << " master? " + << m_AggregatorTwoLevelShm.m_IsMasterAggregator + << " aggr size = " << m_AggregatorTwoLevelShm.m_Size + << " rank = " << m_AggregatorTwoLevelShm.m_Rank + << " subfile = " << m_AggregatorTwoLevelShm.m_SubStreamIndex + << " type = " << m_Parameters.AggregationType + + << std::endl; + + m_IAmDraining = m_AggregatorTwoLevelShm.m_IsMasterAggregator; + m_IAmWritingData = m_AggregatorTwoLevelShm.m_IsAggregator; + m_IAmWritingDataHeader = m_AggregatorTwoLevelShm.m_IsMasterAggregator; + m_Aggregator = + static_cast(&m_AggregatorTwoLevelShm); } } @@ -463,18 +478,18 @@ void BP5Writer::InitTransports() // /path/name.bp.dir/name.bp.rank m_SubStreamNames = - GetBPSubStreamNames(transportsNames, m_Aggregator.m_SubStreamIndex); + GetBPSubStreamNames(transportsNames, m_Aggregator->m_SubStreamIndex); - if (m_Aggregator.m_IsAggregator) + if (m_IAmDraining) { - // Only aggregators will run draining processes + // Only (master)aggregators will run draining processes if (m_DrainBB) { const std::vector drainTransportNames = m_FileDataManager.GetFilesBaseNames( m_Name, m_IO.m_TransportsParameters); m_DrainSubStreamNames = GetBPSubStreamNames( - drainTransportNames, m_Aggregator.m_SubStreamIndex); + drainTransportNames, m_Aggregator->m_SubStreamIndex); /* start up BB thread */ // m_FileDrainer.SetVerbose( // m_Parameters.BurstBufferVerbose, @@ -512,11 +527,23 @@ void BP5Writer::InitTransports() m_IO.m_TransportsParameters[i]["asynctasks"] = "true"; } } - m_FileDataManager.OpenFiles(m_SubStreamNames, m_OpenMode, - m_IO.m_TransportsParameters, false, - m_Aggregator.m_Comm); - if (m_Aggregator.m_IsAggregator) + if (m_EveryoneWrites) + { + m_FileDataManager.OpenFiles(m_SubStreamNames, m_OpenMode, + m_IO.m_TransportsParameters, false, + m_Aggregator->m_Comm); + } + else + { + if (m_IAmWritingData) + { + m_FileDataManager.OpenFiles(m_SubStreamNames, m_OpenMode, + m_IO.m_TransportsParameters, false); + } + } + + if (m_IAmDraining) { if (m_DrainBB) { @@ -703,7 +730,7 @@ void BP5Writer::InitBPBuffer() * them yet so that Open() can stay free of writing to disk) */ - const uint64_t a = static_cast(m_Aggregator.m_SubStreamIndex); + const uint64_t a = static_cast(m_Aggregator->m_SubStreamIndex); std::vector Assignment = m_Comm.GatherValues(a, 0); if (m_Comm.Rank() == 0) @@ -721,7 +748,7 @@ void BP5Writer::InitBPBuffer() sizeof(Assignment[0]) * Assignment.size()); } - if (m_Aggregator.m_IsAggregator) + if (m_IAmWritingDataHeader) { format::BufferSTL d; MakeHeader(d, "Data", false); diff --git a/source/adios2/engine/bp5/BP5Writer.h b/source/adios2/engine/bp5/BP5Writer.h index 66e9720682..d7078f36f6 100644 --- a/source/adios2/engine/bp5/BP5Writer.h +++ b/source/adios2/engine/bp5/BP5Writer.h @@ -13,6 +13,7 @@ #include "adios2/core/Engine.h" #include "adios2/engine/bp5/BP5Engine.h" #include "adios2/helper/adiosComm.h" +#include "adios2/toolkit/aggregator/mpi/MPIChain.h" #include "adios2/toolkit/aggregator/mpi/MPIShmChain.h" #include "adios2/toolkit/burstbuffer/FileDrainerSingleThread.h" #include "adios2/toolkit/format/bp5/BP5Serializer.h" @@ -93,6 +94,8 @@ class BP5Writer : public BP5Engine, public core::Engine /** Parses parameters from IO SetParameters */ void InitParameters() final; + /** Set up the aggregator */ + void InitAggregator(); /** Parses transports and parameters from IO AddTransport */ void InitTransports() final; /** Allocates memory and starts a PG group */ @@ -154,7 +157,13 @@ class BP5Writer : public BP5Engine, public core::Engine void PerformPutCommon(Variable &variable); /** manages all communication tasks in aggregation */ - aggregator::MPIShmChain m_Aggregator; + aggregator::MPIAggregator *m_Aggregator; // points to one of these below + aggregator::MPIShmChain m_AggregatorTwoLevelShm; + aggregator::MPIChain m_AggregatorEveroneWrites; + bool m_IAmDraining = false; + bool m_EveryoneWrites = false; + bool m_IAmWritingData = false; + bool m_IAmWritingDataHeader = false; private: // updated during WriteMetaData diff --git a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp index ccfa243a28..4d7e0c4b52 100644 --- a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp +++ b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp @@ -49,20 +49,25 @@ void BP5Writer::WriteMyOwnData(format::BufferV::BufferV_iovec DataVec) } } -constexpr uint64_t PAGE_SIZE = 65536; // 64KB void BP5Writer::WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec) { + const aggregator::MPIShmChain *a = + dynamic_cast(m_Aggregator); + ; // new step writing starts at offset m_DataPos on aggregator // others will wait for the position to arrive from the rank below - if (m_Aggregator.m_Comm.Rank() > 0) + if (a->m_Comm.Rank() > 0) { - m_Aggregator.m_Comm.Recv(&m_DataPos, 1, m_Aggregator.m_Comm.Rank() - 1, - 0, "Chain token in BP5Writer::WriteData"); + a->m_Comm.Recv(&m_DataPos, 1, a->m_Comm.Rank() - 1, 0, + "Chain token in BP5Writer::WriteData"); } + // align to PAGE_SIZE + m_DataPos += m_Parameters.FileSystemPageSize - + (m_DataPos % m_Parameters.FileSystemPageSize); m_StartDataPos = m_DataPos; - if (m_Aggregator.m_Comm.Rank() < m_Aggregator.m_Comm.Size() - 1) + if (a->m_Comm.Rank() < a->m_Comm.Size() - 1) { int i = 0; uint64_t nextWriterPos = m_DataPos; @@ -71,15 +76,12 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec) nextWriterPos += DataVec[i].iov_len; i++; } - // align to PAGE_SIZE - nextWriterPos += PAGE_SIZE - (nextWriterPos % PAGE_SIZE); - m_Aggregator.m_Comm.Isend(&nextWriterPos, 1, - m_Aggregator.m_Comm.Rank() + 1, 0, - "Chain token in BP5Writer::WriteData"); + a->m_Comm.Isend(&nextWriterPos, 1, a->m_Comm.Rank() + 1, 0, + "Chain token in BP5Writer::WriteData"); } /* Aggregator starts with writing its own data */ - if (m_Aggregator.m_Comm.Rank() == 0) + if (a->m_Comm.Rank() == 0) { WriteMyOwnData(DataVec); } @@ -101,23 +103,19 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec) i++; } - if (m_Aggregator.m_Comm.Size() > 1) + if (a->m_Comm.Size() > 1) { // at the end, last rank sends back the final data pos to first rank // so it can update its data pos - if (m_Aggregator.m_Comm.Rank() == m_Aggregator.m_Comm.Size() - 1) + if (a->m_Comm.Rank() == a->m_Comm.Size() - 1) { - // align to PAGE_SIZE - m_DataPos += PAGE_SIZE - (m_DataPos % PAGE_SIZE); - m_Aggregator.m_Comm.Isend( - &m_DataPos, 1, 0, 0, - "Final chain token in BP5Writer::WriteData"); + a->m_Comm.Isend(&m_DataPos, 1, 0, 0, + "Final chain token in BP5Writer::WriteData"); } - if (m_Aggregator.m_Comm.Rank() == 0) + if (a->m_Comm.Rank() == 0) { - m_Aggregator.m_Comm.Recv(&m_DataPos, 1, - m_Aggregator.m_Comm.Size() - 1, 0, - "Chain token in BP5Writer::WriteData"); + a->m_Comm.Recv(&m_DataPos, 1, a->m_Comm.Size() - 1, 0, + "Chain token in BP5Writer::WriteData"); } } delete[] DataVec; diff --git a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp index 76604ad5f0..7af472fc72 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.cpp @@ -19,14 +19,7 @@ namespace aggregator MPIAggregator::MPIAggregator() {} -MPIAggregator::~MPIAggregator() -{ - if (m_IsActive) - { - m_Comm.Free("freeing aggregators comm in MPIAggregator " - "destructor, not recommended"); - } -} +MPIAggregator::~MPIAggregator() { Close(); } void MPIAggregator::Init(const size_t numAggregators, const size_t subStreams, helper::Comm const &parentComm) @@ -36,15 +29,6 @@ void MPIAggregator::Init(const size_t numAggregators, const size_t subStreams, m_SubStreams = subStreams; } -void MPIAggregator::SwapBuffers(const int step) noexcept {} - -void MPIAggregator::ResetBuffers() noexcept {} - -format::Buffer &MPIAggregator::GetConsumerBuffer(format::Buffer &buffer) -{ - return buffer; -} - void MPIAggregator::Close() { if (m_IsActive) diff --git a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h index 201181e665..67f2b08770 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIAggregator.h @@ -65,39 +65,8 @@ class MPIAggregator virtual void Init(const size_t numAggregators, const size_t subStreams, helper::Comm const &parentComm); - struct ExchangeRequests - { - helper::Comm::Req m_SendSize; - helper::Comm::Req m_SendData; - helper::Comm::Req m_RecvData; - }; - - virtual ExchangeRequests IExchange(format::Buffer &buffer, - const int step) = 0; - - struct ExchangeAbsolutePositionRequests - { - helper::Comm::Req m_Send; - helper::Comm::Req m_Recv; - }; - - virtual ExchangeAbsolutePositionRequests - IExchangeAbsolutePosition(format::Buffer &buffer, const int step) = 0; - - virtual void - WaitAbsolutePosition(ExchangeAbsolutePositionRequests &requests, - const int step) = 0; - - virtual void Wait(ExchangeRequests &requests, const int step) = 0; - - virtual void SwapBuffers(const int step) noexcept; - - virtual void ResetBuffers() noexcept; - - virtual format::Buffer &GetConsumerBuffer(format::Buffer &buffer); - /** closes current aggregator, frees m_Comm */ - void Close(); + virtual void Close(); protected: /** Init m_Comm splitting assigning ranks to subStreams (balanced except for diff --git a/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp index 1bd1e298fa..b81be2ba7b 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIChain.cpp @@ -42,6 +42,8 @@ void MPIChain::Init(const size_t numAggregators, const size_t subStreams, } } +void MPIChain::Close() { MPIAggregator::Close(); } + MPIChain::ExchangeRequests MPIChain::IExchange(format::Buffer &buffer, const int step) { diff --git a/source/adios2/toolkit/aggregator/mpi/MPIChain.h b/source/adios2/toolkit/aggregator/mpi/MPIChain.h index 6cc1f8a288..9a8570c243 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIChain.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIChain.h @@ -30,21 +30,36 @@ class MPIChain : public MPIAggregator void Init(const size_t numAggregators, const size_t subStreams, helper::Comm const &parentComm) final; - ExchangeRequests IExchange(format::Buffer &buffer, const int step) final; + void Close() final; - ExchangeAbsolutePositionRequests - IExchangeAbsolutePosition(format::Buffer &buffer, const int step) final; + struct ExchangeRequests + { + helper::Comm::Req m_SendSize; + helper::Comm::Req m_SendData; + helper::Comm::Req m_RecvData; + }; + + ExchangeRequests IExchange(format::Buffer &buffer, const int step); - void Wait(ExchangeRequests &requests, const int step) final; + struct ExchangeAbsolutePositionRequests + { + helper::Comm::Req m_Send; + helper::Comm::Req m_Recv; + }; + + ExchangeAbsolutePositionRequests + IExchangeAbsolutePosition(format::Buffer &buffer, const int step); void WaitAbsolutePosition(ExchangeAbsolutePositionRequests &requests, - const int step) final; + const int step); + + void Wait(ExchangeRequests &requests, const int step); - void SwapBuffers(const int step) noexcept final; + void SwapBuffers(const int step) noexcept; - void ResetBuffers() noexcept final; + void ResetBuffers() noexcept; - format::Buffer &GetConsumerBuffer(format::Buffer &buffer) final; + format::Buffer &GetConsumerBuffer(format::Buffer &buffer); private: bool m_IsInExchangeAbsolutePosition = false; diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp index f85b7eade3..b628b93eaf 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp @@ -17,6 +17,20 @@ namespace aggregator MPIShmChain::MPIShmChain() : MPIAggregator() {} MPIShmChain::~MPIShmChain() +{ + Close(); + /*if (m_IsActive) + { + m_NodeComm.Free("free per-node comm in ~MPIShmChain()"); + m_OnePerNodeComm.Free("free chain of nodes in ~MPIShmChain()"); + m_AllAggregatorsComm.Free( + "free comm of all aggregators in ~MPIShmChain()"); + m_AggregatorChainComm.Free( + "free chains of aggregators in ~MPIShmChain()"); + }*/ +} + +void MPIShmChain::Close() { if (m_IsActive) { @@ -27,6 +41,7 @@ MPIShmChain::~MPIShmChain() m_AggregatorChainComm.Free( "free chains of aggregators in ~MPIShmChain()"); } + MPIAggregator::Close(); } size_t MPIShmChain::PreInit(helper::Comm const &parentComm) @@ -178,31 +193,5 @@ void MPIShmChain::HandshakeLinks() } } -/*********************** - * Remove these - ***********************/ - -MPIShmChain::ExchangeRequests MPIShmChain::IExchange(format::Buffer &buffer, - const int step) -{ - ExchangeRequests requests; - return requests; -} - -MPIShmChain::ExchangeAbsolutePositionRequests -MPIShmChain::IExchangeAbsolutePosition(format::Buffer &buffer, const int step) -{ - ExchangeAbsolutePositionRequests requests; - return requests; -} - -void MPIShmChain::Wait(ExchangeRequests &requests, const int step) { return; } - -void MPIShmChain::WaitAbsolutePosition( - ExchangeAbsolutePositionRequests &requests, const int step) -{ - return; -} - } // end namespace aggregator } // end namespace adios2 diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h index 04139e6325..de4ace779d 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h @@ -44,6 +44,8 @@ class MPIShmChain : public MPIAggregator void Init(const size_t numAggregators, const size_t subStreams, helper::Comm const &parentComm) final; + void Close() final; + /** * true: the Master (aggregator) process in the chain * always m_Rank == m_Comm.Rank() == 0 for a master aggregator @@ -51,17 +53,6 @@ class MPIShmChain : public MPIAggregator */ bool m_IsMasterAggregator = true; - /* These are not used and must be removed */ - ExchangeRequests IExchange(format::Buffer &buffer, const int step) final; - - ExchangeAbsolutePositionRequests - IExchangeAbsolutePosition(format::Buffer &buffer, const int step) final; - - void Wait(ExchangeRequests &requests, const int step) final; - - void WaitAbsolutePosition(ExchangeAbsolutePositionRequests &requests, - const int step) final; - private: void HandshakeLinks(); From 03437d99b94d965e45a987875b200f9c9f44db4f Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Tue, 13 Jul 2021 12:06:36 -0400 Subject: [PATCH 06/18] two-level aggregation: code skeleton, aggregators and master aggregators in place but they only write their own data --- source/adios2/engine/bp5/BP5Writer.cpp | 28 ++-- source/adios2/engine/bp5/BP5Writer.h | 5 +- .../engine/bp5/BP5Writer_TwoLevelShm.cpp | 127 +++++++++++------- source/adios2/helper/adiosMemory.cpp | 10 ++ source/adios2/helper/adiosMemory.h | 4 + .../toolkit/aggregator/mpi/MPIShmChain.cpp | 4 +- .../toolkit/aggregator/mpi/MPIShmChain.h | 6 +- 7 files changed, 120 insertions(+), 64 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index 48ceddeb7b..d82455793d 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -137,13 +137,13 @@ void BP5Writer::WriteData(format::BufferV *Data) switch (m_Parameters.AggregationType) { case (int)AggregationType::EveryoneWrites: - WriteData_EveryoneWrites(DataVec, false); + WriteData_EveryoneWrites(Data, false); break; case (int)AggregationType::EveryoneWritesSerial: - WriteData_EveryoneWrites(DataVec, true); + WriteData_EveryoneWrites(Data, true); break; case (int)AggregationType::TwoLevelShm: - WriteData_TwoLevelShm(DataVec); + WriteData_TwoLevelShm(Data); break; default: throw std::invalid_argument( @@ -153,12 +153,14 @@ void BP5Writer::WriteData(format::BufferV *Data) } } -void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, +void BP5Writer::WriteData_EveryoneWrites(format::BufferV *Data, bool SerializedWriters) { const aggregator::MPIChain *a = dynamic_cast(m_Aggregator); + format::BufferV::BufferV_iovec DataVec = Data->DataVec(); + // new step writing starts at offset m_DataPos on aggregator // others will wait for the position to arrive from the rank below @@ -169,8 +171,8 @@ void BP5Writer::WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, } // align to PAGE_SIZE - m_DataPos += m_Parameters.FileSystemPageSize - - (m_DataPos % m_Parameters.FileSystemPageSize); + m_DataPos += helper::PaddingToAlignOffset(m_DataPos, + m_Parameters.FileSystemPageSize); m_StartDataPos = m_DataPos; if (!SerializedWriters && a->m_Comm.Rank() < a->m_Comm.Size() - 1) @@ -402,6 +404,16 @@ void BP5Writer::InitParameters() { m_Parameters.NumSubFiles = m_Parameters.NumAggregators; } + + if (m_Parameters.FileSystemPageSize == 0) + { + m_Parameters.FileSystemPageSize = 65536; + } + if (m_Parameters.FileSystemPageSize > 67108864) + { + // Limiting to max 64MB page size + m_Parameters.FileSystemPageSize = 67108864; + } } void BP5Writer::InitAggregator() @@ -748,13 +760,13 @@ void BP5Writer::InitBPBuffer() sizeof(Assignment[0]) * Assignment.size()); } - if (m_IAmWritingDataHeader) + /*if (m_IAmWritingDataHeader) { format::BufferSTL d; MakeHeader(d, "Data", false); m_FileDataManager.WriteFiles(d.m_Buffer.data(), d.m_Position); m_DataPos = d.m_Position; - } + }*/ if (m_Comm.Rank() == 0) { diff --git a/source/adios2/engine/bp5/BP5Writer.h b/source/adios2/engine/bp5/BP5Writer.h index d7078f36f6..28ac3ca742 100644 --- a/source/adios2/engine/bp5/BP5Writer.h +++ b/source/adios2/engine/bp5/BP5Writer.h @@ -13,6 +13,7 @@ #include "adios2/core/Engine.h" #include "adios2/engine/bp5/BP5Engine.h" #include "adios2/helper/adiosComm.h" +#include "adios2/helper/adiosMemory.h" // PaddingToAlignOffset #include "adios2/toolkit/aggregator/mpi/MPIChain.h" #include "adios2/toolkit/aggregator/mpi/MPIShmChain.h" #include "adios2/toolkit/burstbuffer/FileDrainerSingleThread.h" @@ -130,9 +131,9 @@ class BP5Writer : public BP5Engine, public core::Engine /** Write Data to disk, in an aggregator chain */ void WriteData(format::BufferV *Data); - void WriteData_EveryoneWrites(format::BufferV::BufferV_iovec DataVec, + void WriteData_EveryoneWrites(format::BufferV *Data, bool SerializedWriters); - void WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec); + void WriteData_TwoLevelShm(format::BufferV *Data); void PopulateMetadataIndexFileContent( format::BufferSTL &buffer, const uint64_t currentStep, diff --git a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp index 4d7e0c4b52..b1b98d7667 100644 --- a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp +++ b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp @@ -49,75 +49,104 @@ void BP5Writer::WriteMyOwnData(format::BufferV::BufferV_iovec DataVec) } } -void BP5Writer::WriteData_TwoLevelShm(format::BufferV::BufferV_iovec DataVec) +void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) { const aggregator::MPIShmChain *a = dynamic_cast(m_Aggregator); - ; - // new step writing starts at offset m_DataPos on aggregator - // others will wait for the position to arrive from the rank below - if (a->m_Comm.Rank() > 0) - { - a->m_Comm.Recv(&m_DataPos, 1, a->m_Comm.Rank() - 1, 0, - "Chain token in BP5Writer::WriteData"); - } - // align to PAGE_SIZE - m_DataPos += m_Parameters.FileSystemPageSize - - (m_DataPos % m_Parameters.FileSystemPageSize); - m_StartDataPos = m_DataPos; + format::BufferV::BufferV_iovec DataVec = Data->DataVec(); - if (a->m_Comm.Rank() < a->m_Comm.Size() - 1) - { - int i = 0; - uint64_t nextWriterPos = m_DataPos; - while (DataVec[i].iov_base != NULL) - { - nextWriterPos += DataVec[i].iov_len; - i++; - } - a->m_Comm.Isend(&nextWriterPos, 1, a->m_Comm.Rank() + 1, 0, - "Chain token in BP5Writer::WriteData"); - } + // new step writing starts at offset m_DataPos on master aggregator + // other aggregators to the same file will need to wait for the position + // to arrive from the rank below + + // align to PAGE_SIZE (only valid on master aggregator at this point) + m_DataPos += helper::PaddingToAlignOffset(m_DataPos, + m_Parameters.FileSystemPageSize); - /* Aggregator starts with writing its own data */ - if (a->m_Comm.Rank() == 0) + // Each aggregator needs to know the total size they write + // including alignment to page size + // This calculation is valid on aggregators only + std::vector mySizes = a->m_Comm.GatherValues(Data->Size()); + uint64_t myTotalSize = 0; + uint64_t pos = m_DataPos; + for (auto s : mySizes) { - WriteMyOwnData(DataVec); + uint64_t alignment = + helper::PaddingToAlignOffset(pos, m_Parameters.FileSystemPageSize); + myTotalSize += alignment + s; + pos += alignment + s; } - int i = 0; - while (DataVec[i].iov_base != NULL) + int shmFillerToken = 0; + if (a->m_IsAggregator) { - if (i == 0) + // In each aggregator chain, send from master down the line + // these total sizes, so every aggregator knows where to start + if (a->m_AggregatorChainComm.Rank() > 0) { - m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, - DataVec[i].iov_len, m_StartDataPos); + a->m_AggregatorChainComm.Recv( + &m_DataPos, 1, a->m_AggregatorChainComm.Rank() - 1, 0, + "AggregatorChain token in BP5Writer::WriteData_TwoLevelShm"); + // align to PAGE_SIZE + m_DataPos += helper::PaddingToAlignOffset( + m_DataPos, m_Parameters.FileSystemPageSize); } - else + if (a->m_AggregatorChainComm.Rank() < + a->m_AggregatorChainComm.Size() - 1) { - m_FileDataManager.WriteFiles((char *)DataVec[i].iov_base, - DataVec[i].iov_len); + uint64_t nextWriterPos = m_DataPos + myTotalSize; + a->m_AggregatorChainComm.Isend( + &nextWriterPos, 1, a->m_AggregatorChainComm.Rank() + 1, 0, + "Chain token in BP5Writer::WriteData"); } - m_DataPos += DataVec[i].iov_len; - i++; - } + else if (a->m_AggregatorChainComm.Size() > 1) + { + // send back final position from last aggregator in file to master + // aggregator + uint64_t nextWriterPos = m_DataPos + myTotalSize; + a->m_AggregatorChainComm.Isend( + &nextWriterPos, 1, 0, 0, "Chain token in BP5Writer::WriteData"); + } + std::cout << "Rank " << m_Comm.Rank() << " aggregator writes step " + << m_WriterStep << " to subfile " << a->m_SubStreamIndex + << " at pos " << m_DataPos << " size " << myTotalSize + << std::endl; + // Send token to first non-aggregator to start filling shm + if (a->m_Comm.Size() > 1) + { + a->m_Comm.Isend(&shmFillerToken, 1, a->m_Comm.Rank() + 1, 0, + "Shm token in BP5Writer::WriteData_TwoLevelShm"); + } + WriteMyOwnData(DataVec); - if (a->m_Comm.Size() > 1) - { - // at the end, last rank sends back the final data pos to first rank - // so it can update its data pos - if (a->m_Comm.Rank() == a->m_Comm.Size() - 1) + /* TODO Write from shm until it's over */ + + // Master aggregator needs to know where the last writing ended by the + // last aggregator in the chain, so that it can start from the correct + // position at the next output step + if (a->m_AggregatorChainComm.Size() > 1 && + !a->m_AggregatorChainComm.Rank()) { - a->m_Comm.Isend(&m_DataPos, 1, 0, 0, - "Final chain token in BP5Writer::WriteData"); + a->m_AggregatorChainComm.Recv( + &m_DataPos, 1, a->m_AggregatorChainComm.Size() - 1, 0, + "Chain token in BP5Writer::WriteData"); } - if (a->m_Comm.Rank() == 0) + } + else + { + // non-aggregators fill shared buffer in marching order + a->m_Comm.Recv(&shmFillerToken, 1, a->m_Comm.Rank() - 1, 0, + "Shm token in BP5Writer::WriteData_TwoLevelShm"); + std::cout << "Rank " << m_Comm.Rank() + << " non-aggregator recv token to fill shm " << std::endl; + if (a->m_Comm.Rank() < a->m_Comm.Size() - 1) { - a->m_Comm.Recv(&m_DataPos, 1, a->m_Comm.Size() - 1, 0, - "Chain token in BP5Writer::WriteData"); + a->m_Comm.Isend(&shmFillerToken, 1, a->m_Comm.Rank() + 1, 0, + "Shm token in BP5Writer::WriteData_TwoLevelShm"); } } + delete[] DataVec; } diff --git a/source/adios2/helper/adiosMemory.cpp b/source/adios2/helper/adiosMemory.cpp index 3bf8c89adc..549c2c3af4 100644 --- a/source/adios2/helper/adiosMemory.cpp +++ b/source/adios2/helper/adiosMemory.cpp @@ -297,5 +297,15 @@ size_t PaddingToAlignPointer(const void *ptr) return padSize; } +uint64_t PaddingToAlignOffset(uint64_t offset, uint64_t alignment_size) +{ + uint64_t padSize = alignment_size - (offset % alignment_size); + if (padSize == alignment_size) + { + padSize = 0; + } + return padSize; +} + } // end namespace helper } // end namespace adios2 diff --git a/source/adios2/helper/adiosMemory.h b/source/adios2/helper/adiosMemory.h index 9d5e40fbd5..009cf1b318 100644 --- a/source/adios2/helper/adiosMemory.h +++ b/source/adios2/helper/adiosMemory.h @@ -243,6 +243,10 @@ size_t PayloadSize(const T *data, const Dims &count) noexcept; */ size_t PaddingToAlignPointer(const void *ptr); +/** Calculate padding to an arbitrary offset to be aligned to + * the size alignment_size */ +uint64_t PaddingToAlignOffset(uint64_t offset, uint64_t alignment_size); + } // end namespace helper } // end namespace adios2 diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp index b628b93eaf..8293e67e91 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp @@ -85,7 +85,7 @@ void MPIShmChain::Init(const size_t numAggregators, const size_t subStreams, size_t aggregatorPerNode = numAggregators / m_NumNodes; if (aggregatorPerNode == 0) { - aggregatorPerNode = 1; + aggregatorPerNode = 1; /* default */ } if (aggregatorPerNode > NodeSize) { @@ -130,7 +130,7 @@ void MPIShmChain::Init(const size_t numAggregators, const size_t subStreams, m_SubStreams = subStreams; if (m_SubStreams == 0) { - m_SubStreams = 1; + m_SubStreams = m_NumAggregators; /* default */ } if (m_SubStreams > m_NumAggregators) { diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h index de4ace779d..d539bdf4dd 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h @@ -53,9 +53,6 @@ class MPIShmChain : public MPIAggregator */ bool m_IsMasterAggregator = true; -private: - void HandshakeLinks(); - /* Variables set in PreInit */ @@ -87,6 +84,9 @@ class MPIShmChain : public MPIAggregator Useful only on aggregators themselves */ helper::Comm m_AggregatorChainComm; + +private: + void HandshakeLinks(); }; } // end namespace aggregator From 9f8916c556e188657a2d08c6d826f872e87426d1 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Wed, 14 Jul 2021 15:22:04 -0400 Subject: [PATCH 07/18] Add some Win_ functions to Comm, to use MPI's shared memory functionality --- source/adios2/helper/adiosComm.cpp | 55 ++++++++++++++ source/adios2/helper/adiosComm.h | 95 +++++++++++++++++++++++++ source/adios2/helper/adiosCommDummy.cpp | 58 +++++++++++++++ source/adios2/helper/adiosCommMPI.cpp | 82 +++++++++++++++++++++ 4 files changed, 290 insertions(+) diff --git a/source/adios2/helper/adiosComm.cpp b/source/adios2/helper/adiosComm.cpp index 42ce4d7fce..4c119f65cc 100644 --- a/source/adios2/helper/adiosComm.cpp +++ b/source/adios2/helper/adiosComm.cpp @@ -114,6 +114,31 @@ std::vector Comm::GetGathervDisplacements(const size_t *counts, return displacements; } +Comm::Win Comm::Win_allocate_shared(size_t size, int disp_unit, void *baseptr, + const std::string &hint) +{ + return m_Impl->Win_allocate_shared(size, disp_unit, baseptr, hint); +} +int Comm::Win_shared_query(Comm::Win &win, int rank, size_t *size, + int *disp_unit, void *baseptr, + const std::string &hint) +{ + return m_Impl->Win_shared_query(win, rank, size, disp_unit, baseptr, hint); +} +int Comm::Win_free(Win &win, const std::string &hint) +{ + return m_Impl->Win_free(win, hint); +} +int Comm::Win_Lock(LockType lock_type, int rank, int assert, Win &win, + const std::string &hint) +{ + return m_Impl->Win_Lock(lock_type, rank, assert, win, hint); +} +int Comm::Win_Unlock(int rank, Win &win, const std::string &hint) +{ + return m_Impl->Win_Unlock(rank, win, hint); +} + Comm::Req::Req() = default; Comm::Req::Req(std::unique_ptr impl) : m_Impl(std::move(impl)) {} @@ -135,6 +160,27 @@ Comm::Status Comm::Req::Wait(const std::string &hint) return status; } +Comm::Win::Win() = default; + +Comm::Win::Win(std::unique_ptr impl) : m_Impl(std::move(impl)) {} + +Comm::Win::~Win() = default; + +Comm::Win::Win(Win &&win) = default; + +Comm::Win &Comm::Win::operator=(Win &&win) = default; + +int Comm::Win::Free(const std::string &hint) +{ + int status = 0; + if (m_Impl) + { + status = m_Impl->Free(hint); + m_Impl.reset(); + } + return status; +} + CommImpl::~CommImpl() = default; size_t CommImpl::SizeOf(Datatype datatype) { return ToSize(datatype); } @@ -149,9 +195,18 @@ Comm::Req CommImpl::MakeReq(std::unique_ptr impl) return Comm::Req(std::move(impl)); } +Comm::Win CommImpl::MakeWin(std::unique_ptr impl) +{ + return Comm::Win(std::move(impl)); +} + CommImpl *CommImpl::Get(Comm const &comm) { return comm.m_Impl.get(); } +CommWinImpl *CommWinImpl::Get(Comm::Win const &win) { return win.m_Impl.get(); } + CommReqImpl::~CommReqImpl() = default; +CommWinImpl::~CommWinImpl() = default; + } // end namespace helper } // end namespace adios2 diff --git a/source/adios2/helper/adiosComm.h b/source/adios2/helper/adiosComm.h index f8e2c7cf9e..4f4ed2fc54 100644 --- a/source/adios2/helper/adiosComm.h +++ b/source/adios2/helper/adiosComm.h @@ -19,6 +19,7 @@ namespace helper class CommImpl; class CommReqImpl; +class CommWinImpl; /** @brief Encapsulation for communication in a multi-process environment. */ class Comm @@ -26,6 +27,7 @@ class Comm public: class Req; class Status; + class Win; /** * @brief Enumeration of element-wise accumulation operations. @@ -49,6 +51,15 @@ class Comm None, }; + /** + * @brief Enumeration of locking operations. + */ + enum class LockType + { + Exclusive, + Shared + }; + /** * @brief Default constructor. Produces an empty communicator. * @@ -254,6 +265,16 @@ class Comm Req Irecv(T *buffer, const size_t count, int source, int tag, const std::string &hint = std::string()) const; + Win Win_allocate_shared(size_t size, int disp_unit, void *baseptr, + const std::string &hint = std::string()); + int Win_shared_query(Win &win, int rank, size_t *size, int *disp_unit, + void *baseptr, + const std::string &hint = std::string()); + int Win_free(Win &win, const std::string &hint = std::string()); + int Win_Lock(LockType lock_type, int rank, int assert, Win &win, + const std::string &hint = std::string()); + int Win_Unlock(int rank, Win &win, const std::string &hint = std::string()); + private: friend class CommImpl; @@ -340,6 +361,59 @@ class Comm::Status bool Cancelled = false; }; +class Comm::Win +{ +public: + /** + * @brief Default constructor. Produces an empty Win. + * + * An empty Win may not be used. + */ + Win(); + + /** + * @brief Move constructor. Moves Win state from that given. + * + * The moved-from Win is left empty and may not be used. + */ + Win(Win &&); + + /** + * @brief Deleted copy constructor. A Win may not be copied. + */ + Win(Win const &) = delete; + + ~Win(); + + /** + * @brief Move assignment. Moves Win state from that given. + * + * The moved-from Win is left empty and may not be used. + */ + Win &operator=(Win &&); + + /** + * @brief Deleted copy assignment. A Win may not be copied. + */ + Win &operator=(Win const &) = delete; + + /** + * @brief Free the Win object. + * + * On return, the Win is empty. + * For an MPI Win object this is equivalent to the call MPI_Win_free() + */ + int Free(const std::string &hint = std::string()); + +private: + friend class CommImpl; + friend class CommWinImpl; + + explicit Win(std::unique_ptr impl); + + std::unique_ptr m_Impl; +}; + class CommImpl { public: @@ -437,10 +511,23 @@ class CommImpl int source, int tag, const std::string &hint) const = 0; + virtual Comm::Win Win_allocate_shared(size_t size, int disp_unit, + void *baseptr, + const std::string &hint) const = 0; + virtual int Win_shared_query(Comm::Win &win, int rank, size_t *size, + int *disp_unit, void *baseptr, + const std::string &hint) const = 0; + virtual int Win_free(Comm::Win &win, const std::string &hint) const = 0; + virtual int Win_Lock(Comm::LockType lock_type, int rank, int assert, + Comm::Win &win, const std::string &hint) const = 0; + virtual int Win_Unlock(int rank, Comm::Win &win, + const std::string &hint) const = 0; + static size_t SizeOf(Datatype datatype); static Comm MakeComm(std::unique_ptr impl); static Comm::Req MakeReq(std::unique_ptr impl); + static Comm::Win MakeWin(std::unique_ptr impl); static CommImpl *Get(Comm const &comm); }; @@ -451,6 +538,14 @@ class CommReqImpl virtual Comm::Status Wait(const std::string &hint) = 0; }; +class CommWinImpl +{ +public: + virtual ~CommWinImpl() = 0; + virtual int Free(const std::string &hint) = 0; + static CommWinImpl *Get(Comm::Win const &win); +}; + } // end namespace helper } // end namespace adios2 diff --git a/source/adios2/helper/adiosCommDummy.cpp b/source/adios2/helper/adiosCommDummy.cpp index eddf9827d5..0af8371bc3 100644 --- a/source/adios2/helper/adiosCommDummy.cpp +++ b/source/adios2/helper/adiosCommDummy.cpp @@ -38,6 +38,17 @@ class CommReqImplDummy : public CommReqImpl CommReqImplDummy::~CommReqImplDummy() = default; +class CommWinImplDummy : public CommWinImpl +{ +public: + CommWinImplDummy() {} + ~CommWinImplDummy() override; + + int Free(const std::string &hint) override; +}; + +CommWinImplDummy::~CommWinImplDummy() = default; + class CommImplDummy : public CommImpl { public: @@ -104,6 +115,16 @@ class CommImplDummy : public CommImpl Comm::Req Irecv(void *buffer, size_t count, Datatype datatype, int source, int tag, const std::string &hint) const override; + + Comm::Win Win_allocate_shared(size_t size, int disp_unit, void *baseptr, + const std::string &hint) const override; + int Win_shared_query(Comm::Win &win, int rank, size_t *size, int *disp_unit, + void *baseptr, const std::string &hint) const override; + int Win_free(Comm::Win &win, const std::string &hint) const override; + int Win_Lock(Comm::LockType lock_type, int rank, int assert, Comm::Win &win, + const std::string &hint) const override; + int Win_Unlock(int rank, Comm::Win &win, + const std::string &hint) const override; }; CommImplDummy::~CommImplDummy() = default; @@ -285,12 +306,49 @@ Comm::Req CommImplDummy::Irecv(void *, size_t, Datatype, int, int, return MakeReq(std::move(req)); } +Comm::Win CommImplDummy::Win_allocate_shared(size_t size, int disp_unit, + void *baseptr, + const std::string &) const +{ + auto win = std::unique_ptr(new CommWinImplDummy()); + baseptr = nullptr; + return MakeWin(std::move(win)); +} + +int CommImplDummy::Win_shared_query(Comm::Win &win, int rank, size_t *size, + int *disp_unit, void *baseptr, + const std::string &) const +{ + *size = 0; + *disp_unit = 1; + baseptr = nullptr; + return 0; +} + +int CommImplDummy::Win_free(Comm::Win &win, const std::string &) const +{ + win.Free(); +} + +int CommImplDummy::Win_Lock(Comm::LockType lock_type, int rank, int assert, + Comm::Win &win, const std::string &) const +{ + return 0; +} +int CommImplDummy::Win_Unlock(int rank, Comm::Win &win, + const std::string &) const +{ + return 0; +} + Comm::Status CommReqImplDummy::Wait(const std::string &hint) { Comm::Status status; return status; } +int CommWinImplDummy::Free(const std::string &hint) { return 0; } + Comm CommDummy() { auto comm = std::unique_ptr(new CommImplDummy()); diff --git a/source/adios2/helper/adiosCommMPI.cpp b/source/adios2/helper/adiosCommMPI.cpp index eebc4e7251..0ccf1e39e3 100644 --- a/source/adios2/helper/adiosCommMPI.cpp +++ b/source/adios2/helper/adiosCommMPI.cpp @@ -43,6 +43,10 @@ const MPI_Op OpToMPI[] = { MPI_Op ToMPI(Comm::Op op) { return OpToMPI[int(op)]; } +const int LockTypeToMPI[] = {MPI_LOCK_EXCLUSIVE, MPI_LOCK_SHARED}; + +int ToMPI(Comm::LockType lock_type) { return LockTypeToMPI[int(lock_type)]; } + const MPI_Datatype DatatypeToMPI[] = { MPI_SIGNED_CHAR, MPI_CHAR, @@ -108,6 +112,19 @@ class CommReqImplMPI : public CommReqImpl CommReqImplMPI::~CommReqImplMPI() = default; +class CommWinImplMPI : public CommWinImpl +{ +public: + CommWinImplMPI() {} + ~CommWinImplMPI() override; + + int Free(const std::string &hint) override; + + MPI_Win m_Win; +}; + +CommWinImplMPI::~CommWinImplMPI() = default; + class CommImplMPI : public CommImpl { public: @@ -177,6 +194,16 @@ class CommImplMPI : public CommImpl Comm::Req Irecv(void *buffer, size_t count, Datatype datatype, int source, int tag, const std::string &hint) const override; + + Comm::Win Win_allocate_shared(size_t size, int disp_unit, void *baseptr, + const std::string &hint) const override; + int Win_shared_query(Comm::Win &win, int rank, size_t *size, int *disp_unit, + void *baseptr, const std::string &hint) const override; + int Win_free(Comm::Win &win, const std::string &hint) const override; + int Win_Lock(Comm::LockType lock_type, int rank, int assert, Comm::Win &win, + const std::string &hint) const override; + int Win_Unlock(int rank, Comm::Win &win, + const std::string &hint) const override; }; CommImplMPI::~CommImplMPI() @@ -519,6 +546,56 @@ Comm::Req CommImplMPI::Irecv(void *buffer, size_t count, Datatype datatype, return MakeReq(std::move(req)); } +Comm::Win CommImplMPI::Win_allocate_shared(size_t size, int disp_unit, + void *baseptr, + const std::string &hint) const +{ + auto w = std::unique_ptr(new CommWinImplMPI()); + MPI_Aint asize = static_cast(size); + CheckMPIReturn(MPI_Win_allocate_shared(asize, disp_unit, MPI_INFO_NULL, + m_MPIComm, baseptr, &w->m_Win), + "in call to Win_allocate_shared " + hint + "\n"); + return MakeWin(std::move(w)); +} + +int CommImplMPI::Win_shared_query(Comm::Win &win, int rank, size_t *size, + int *disp_unit, void *baseptr, + const std::string &hint) const +{ + CommWinImplMPI *w = dynamic_cast(CommWinImpl::Get(win)); + MPI_Aint asize; + int ret = MPI_Win_shared_query(w->m_Win, rank, &asize, disp_unit, baseptr); + CheckMPIReturn(ret, "in call to Win_shared_query " + hint + "\n"); + *size = static_cast(asize); + return ret; +} + +int CommImplMPI::Win_free(Comm::Win &win, const std::string &hint) const +{ + CommWinImplMPI *w = dynamic_cast(CommWinImpl::Get(win)); + int ret = MPI_Win_free(&w->m_Win); + CheckMPIReturn(ret, "in call to Win_free " + hint + "\n"); + return ret; +} + +int CommImplMPI::Win_Lock(Comm::LockType lock_type, int rank, int assert, + Comm::Win &win, const std::string &hint) const +{ + CommWinImplMPI *w = dynamic_cast(CommWinImpl::Get(win)); + int mpi_lock_type = ToMPI(lock_type); + int ret = MPI_Win_lock(mpi_lock_type, rank, assert, w->m_Win); + CheckMPIReturn(ret, "in call to Win_Lock " + hint + "\n"); + return ret; +} +int CommImplMPI::Win_Unlock(int rank, Comm::Win &win, + const std::string &hint) const +{ + CommWinImplMPI *w = dynamic_cast(CommWinImpl::Get(win)); + int ret = MPI_Win_unlock(rank, w->m_Win); + CheckMPIReturn(ret, "in call to Win_Lock " + hint + "\n"); + return ret; +} + Comm::Status CommReqImplMPI::Wait(const std::string &hint) { Comm::Status status; @@ -580,6 +657,11 @@ Comm::Status CommReqImplMPI::Wait(const std::string &hint) return status; } +int CommWinImplMPI::Free(const std::string &hint) +{ + return MPI_Win_free(&m_Win); +} + Comm CommWithMPI(MPI_Comm mpiComm) { static InitMPI const initMPI; From 57941f1844d57f4d1ad1568e3ed232df640bb12f Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Wed, 14 Jul 2021 15:23:24 -0400 Subject: [PATCH 08/18] Allocate a shared memory segment for each aggregator with two separate buffers A and B, which are filled by non-aggregators (producers) while the aggregator (consumer) writes data to disk. Unfinished work... --- source/adios2/engine/bp5/BP5Writer.h | 5 +- .../engine/bp5/BP5Writer_TwoLevelShm.cpp | 186 +++++++++++++++--- .../toolkit/aggregator/mpi/MPIShmChain.cpp | 160 ++++++++++++++- .../toolkit/aggregator/mpi/MPIShmChain.h | 72 ++++++- 4 files changed, 388 insertions(+), 35 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Writer.h b/source/adios2/engine/bp5/BP5Writer.h index 28ac3ca742..9673c4ab87 100644 --- a/source/adios2/engine/bp5/BP5Writer.h +++ b/source/adios2/engine/bp5/BP5Writer.h @@ -147,8 +147,11 @@ class BP5Writer : public BP5Engine, public core::Engine void MarshalAttributes(); - /* Shmem aggregator functions */ + /* Two-level-shm aggregator functions */ void WriteMyOwnData(format::BufferV::BufferV_iovec DataVec); + void SendDataToAggregator(format::BufferV::BufferV_iovec DataVec, + const size_t TotalSize); + void WriteOthersData(const size_t TotalSize); template T *BufferDataCommon(const size_t payloadOffset, diff --git a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp index b1b98d7667..f7c12e5fac 100644 --- a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp +++ b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp @@ -28,27 +28,6 @@ namespace engine using namespace adios2::format; -void BP5Writer::WriteMyOwnData(format::BufferV::BufferV_iovec DataVec) -{ - m_StartDataPos = m_DataPos; - int i = 0; - while (DataVec[i].iov_base != NULL) - { - if (i == 0) - { - m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, - DataVec[i].iov_len, m_StartDataPos); - } - else - { - m_FileDataManager.WriteFiles((char *)DataVec[i].iov_base, - DataVec[i].iov_len); - } - m_DataPos += DataVec[i].iov_len; - i++; - } -} - void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) { const aggregator::MPIShmChain *a = @@ -64,6 +43,7 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) m_DataPos += helper::PaddingToAlignOffset(m_DataPos, m_Parameters.FileSystemPageSize); + /* // Each aggregator needs to know the total size they write // including alignment to page size // This calculation is valid on aggregators only @@ -77,8 +57,17 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) myTotalSize += alignment + s; pos += alignment + s; } + */ + + // Each aggregator needs to know the total size they write + // This calculation is valid on aggregators only + std::vector mySizes = a->m_Comm.GatherValues(Data->Size()); + uint64_t myTotalSize = 0; + for (auto s : mySizes) + { + myTotalSize += s; + } - int shmFillerToken = 0; if (a->m_IsAggregator) { // In each aggregator chain, send from master down the line @@ -92,6 +81,7 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) m_DataPos += helper::PaddingToAlignOffset( m_DataPos, m_Parameters.FileSystemPageSize); } + m_StartDataPos = m_DataPos; // metadata needs this info if (a->m_AggregatorChainComm.Rank() < a->m_AggregatorChainComm.Size() - 1) { @@ -108,19 +98,26 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) a->m_AggregatorChainComm.Isend( &nextWriterPos, 1, 0, 0, "Chain token in BP5Writer::WriteData"); } - std::cout << "Rank " << m_Comm.Rank() << " aggregator writes step " - << m_WriterStep << " to subfile " << a->m_SubStreamIndex - << " at pos " << m_DataPos << " size " << myTotalSize - << std::endl; + std::cout << "Rank " << m_Comm.Rank() + << " aggregator start writing step " << m_WriterStep + << " to subfile " << a->m_SubStreamIndex << " at pos " + << m_DataPos << " totalsize " << myTotalSize << std::endl; // Send token to first non-aggregator to start filling shm + // Also informs next process its starting offset (for correct metadata) if (a->m_Comm.Size() > 1) { - a->m_Comm.Isend(&shmFillerToken, 1, a->m_Comm.Rank() + 1, 0, + uint64_t nextWriterPos = m_DataPos + Data->Size(); + a->m_Comm.Isend(&nextWriterPos, 1, a->m_Comm.Rank() + 1, 0, "Shm token in BP5Writer::WriteData_TwoLevelShm"); } + WriteMyOwnData(DataVec); - /* TODO Write from shm until it's over */ + /* Write from shm until every non-aggr sent all data */ + if (a->m_Comm.Size() > 1) + { + WriteOthersData(myTotalSize - Data->Size()); + } // Master aggregator needs to know where the last writing ended by the // last aggregator in the chain, so that it can start from the correct @@ -136,13 +133,19 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) else { // non-aggregators fill shared buffer in marching order - a->m_Comm.Recv(&shmFillerToken, 1, a->m_Comm.Rank() - 1, 0, + // they also receive their starting offset this way + a->m_Comm.Recv(&m_StartDataPos, 1, a->m_Comm.Rank() - 1, 0, "Shm token in BP5Writer::WriteData_TwoLevelShm"); std::cout << "Rank " << m_Comm.Rank() - << " non-aggregator recv token to fill shm " << std::endl; + << " non-aggregator recv token to fill shm = " + << m_StartDataPos << std::endl; + + SendDataToAggregator(DataVec, Data->Size()); + if (a->m_Comm.Rank() < a->m_Comm.Size() - 1) { - a->m_Comm.Isend(&shmFillerToken, 1, a->m_Comm.Rank() + 1, 0, + uint64_t nextWriterPos = m_StartDataPos + Data->Size(); + a->m_Comm.Isend(&nextWriterPos, 1, a->m_Comm.Rank() + 1, 0, "Shm token in BP5Writer::WriteData_TwoLevelShm"); } } @@ -150,6 +153,127 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) delete[] DataVec; } +void BP5Writer::WriteMyOwnData(format::BufferV::BufferV_iovec DataVec) +{ + m_StartDataPos = m_DataPos; + int i = 0; + while (DataVec[i].iov_base != NULL) + { + if (i == 0) + { + m_FileDataManager.WriteFileAt((char *)DataVec[i].iov_base, + DataVec[i].iov_len, m_StartDataPos); + } + else + { + m_FileDataManager.WriteFiles((char *)DataVec[i].iov_base, + DataVec[i].iov_len); + } + m_DataPos += DataVec[i].iov_len; + i++; + } +} + +void BP5Writer::SendDataToAggregator(format::BufferV::BufferV_iovec DataVec, + const size_t TotalSize) +{ + /* Only one process is running this function at once + See shmFillerToken in the caller function + + In a loop, copy the local data into the shared memory, alternating + between the two segments. + */ + + aggregator::MPIShmChain *a = + dynamic_cast(m_Aggregator); + + size_t sent = 0; + int block = 0; + size_t temp_offset = 0; + while (DataVec[block].iov_base != nullptr) + { + // potentially blocking call waiting on Aggregator + aggregator::MPIShmChain::ShmDataBuffer *b = a->LockProducerBuffer(); + // b->max_size: how much we can copy + // b->actual_size: how much we actually copy + b->actual_size = 0; + while (true) + { + if (DataVec[block].iov_base == nullptr) + { + break; + } + /* Copy n bytes from the current block, current offset to shm + making sure to use up to shm_size bytes + */ + size_t n = DataVec[block].iov_len - temp_offset; + if (n > (b->max_size - b->actual_size)) + { + n = b->max_size - b->actual_size; + } + std::memcpy(&b->buf[b->actual_size], + (const char *)DataVec[block].iov_base + temp_offset, n); + b->actual_size += n; + + /* Have we processed the entire block or staying with it? */ + if (n + temp_offset < DataVec[block].iov_len) + { + temp_offset += n; + } + else + { + temp_offset = 0; + ++block; + } + + /* Have we reached the max allowed shm size ?*/ + if (b->actual_size >= b->max_size) + { + break; + } + } + sent += b->actual_size; + std::cout << "Rank " << m_Comm.Rank() + << " filled shm, data_size = " << b->actual_size + << " block = " << block << " temp offset = " << temp_offset + << " sent = " << sent + << " buf = " << static_cast(b->buf) << " = [" + << (int)b->buf[0] << (int)b->buf[1] << "..." + << (int)b->buf[b->actual_size - 2] + << (int)b->buf[b->actual_size - 1] << "]" << std::endl; + + a->UnlockProducerBuffer(); + } +} +void BP5Writer::WriteOthersData(size_t TotalSize) +{ + /* Only an Aggregator calls this function */ + aggregator::MPIShmChain *a = + dynamic_cast(m_Aggregator); + + size_t wrote = 0; + while (wrote < TotalSize) + { + // potentially blocking call waiting on some non-aggr process + aggregator::MPIShmChain::ShmDataBuffer *b = a->LockConsumerBuffer(); + + std::cout << "Rank " << m_Comm.Rank() + << " write from shm, data_size = " << b->actual_size + << " total so far = " << wrote + << " buf = " << static_cast(b->buf) << " = [" + << (int)b->buf[0] << (int)b->buf[1] << "..." + << (int)b->buf[b->actual_size - 2] + << (int)b->buf[b->actual_size - 1] << "]" << std::endl; + + // b->actual_size: how much we need to write + m_FileDataManager.WriteFiles(b->buf, b->actual_size); + + wrote += b->actual_size; + + a->UnlockConsumerBuffer(); + } +} + } // end namespace engine } // end namespace core } // end namespace adios2 diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp index 8293e67e91..cd3235b7b1 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp @@ -9,6 +9,8 @@ */ #include "MPIShmChain.h" +#include + namespace adios2 { namespace aggregator @@ -160,11 +162,19 @@ void MPIShmChain::Init(const size_t numAggregators, const size_t subStreams, m_IsActive = true; - HandshakeLinks(); + helper::Comm::Req sendRequest = HandshakeLinks_Start(); + + /* Create the shared memory segment */ + if (m_Comm.Size() > 1) + { + CreateShm(); + } + + HandshakeLinks_Complete(sendRequest); } // PRIVATE -void MPIShmChain::HandshakeLinks() +helper::Comm::Req MPIShmChain::HandshakeLinks_Start() { int link = -1; @@ -191,6 +201,152 @@ void MPIShmChain::HandshakeLinks() sendRequest.Wait("Isend wait handshake with neighbor, MPIChain " "aggregator, at Open"); } + return sendRequest; +} + +void MPIShmChain::HandshakeLinks_Complete(helper::Comm::Req &req) +{ + if (m_Rank > 0) + { + req.Wait("Isend wait handshake with neighbor, MPIChain " + "aggregator, at Open"); + } +} + +void MPIShmChain::CreateShm() +{ + void *ptr; + if (!m_Rank) + { + m_Win = m_Comm.Win_allocate_shared(sizeof(ShmSegment), 1, &ptr); + } + + if (m_Rank) + { + m_Win = m_Comm.Win_allocate_shared(0, 1, &ptr); + size_t shmsize; + int disp_unit; + m_Comm.Win_shared_query(m_Win, 0, &shmsize, &disp_unit, &ptr); + } + m_Shm = reinterpret_cast(ptr); + m_Shm->producerBuffer = BufferUse::None; + m_Shm->consumerBuffer = BufferUse::None; + m_Shm->sdbA.buf = nullptr; + m_Shm->sdbA.max_size = SHM_BUF_SIZE; + m_Shm->sdbB.buf = nullptr; + m_Shm->sdbB.max_size = SHM_BUF_SIZE; + + std::cout << "Rank " << m_Rank << " shm = " << ptr + << " bufA = " << static_cast(m_Shm->bufA) + << " bufB = " << static_cast(m_Shm->bufB) << std::endl; +} + +void MPIShmChain::DestroyShm() { m_Comm.Win_free(m_Win); } + +MPIShmChain::ShmDataBuffer *MPIShmChain::LockProducerBuffer() +{ + MPIShmChain::ShmDataBuffer *sdb = nullptr; + + m_Comm.Win_Lock(helper::Comm::LockType::Exclusive, 0, 0, m_Win); + if (m_Shm->producerBuffer == BufferUse::A) + + { + m_Shm->producerBuffer = BufferUse::B; + sdb = &m_Shm->sdbB; + // point to shm data buffer (in local process memory) + sdb->buf = m_Shm->bufB; + } + else // (m_Shm->producerBuffer == BufferUse::None || + // m_Shm->producerBuffer == BufferUse::B) + { + m_Shm->producerBuffer = BufferUse::A; + sdb = &m_Shm->sdbA; + // point to shm data buffer (in local process memory) + sdb->buf = m_Shm->bufA; + } + m_Comm.Win_Unlock(0, m_Win); + + // We determined we want a specific buffer + // Now we need to get a lock on it in case consumer is using it + if (m_Shm->producerBuffer == BufferUse::A) + { + m_Shm->lockA.lock(); + } + else + { + m_Shm->lockB.lock(); + } + + return sdb; +} + +void MPIShmChain::UnlockProducerBuffer() +{ + if (m_Shm->producerBuffer == BufferUse::A) + { + m_Shm->lockA.unlock(); + } + else + { + m_Shm->lockB.unlock(); + } +} + +MPIShmChain::ShmDataBuffer *MPIShmChain::LockConsumerBuffer() +{ + MPIShmChain::ShmDataBuffer *sdb = nullptr; + + // Sleep until the very first production has started: + while (m_Shm->producerBuffer == BufferUse::None) + { + std::this_thread::sleep_for(std::chrono::duration(0.00001)); + } + // At this point we know buffer A has content or going to have content + // when we successfully lock it + + m_Comm.Win_Lock(helper::Comm::LockType::Exclusive, 0, 0, m_Win); + if (m_Shm->consumerBuffer == BufferUse::A) + + { + m_Shm->consumerBuffer = BufferUse::B; + sdb = &m_Shm->sdbB; + // point to shm data buffer (in local process memory) + sdb->buf = m_Shm->bufB; + } + else // (m_Shm->consumerBuffer == BufferUse::None || + // m_Shm->consumerBuffer == BufferUse::B) + { + m_Shm->consumerBuffer = BufferUse::A; + sdb = &m_Shm->sdbA; + // point to shm data buffer (in local process memory) + sdb->buf = m_Shm->bufA; + } + m_Comm.Win_Unlock(0, m_Win); + + // We determined we want a specific buffer + // Now we need to get a lock on it in case producer is using it + if (m_Shm->consumerBuffer == BufferUse::A) + { + m_Shm->lockA.lock(); + } + else + { + m_Shm->lockB.lock(); + } + + return sdb; +} + +void MPIShmChain::UnlockConsumerBuffer() +{ + if (m_Shm->consumerBuffer == BufferUse::A) + { + m_Shm->lockA.unlock(); + } + else + { + m_Shm->lockB.unlock(); + } } } // end namespace aggregator diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h index d539bdf4dd..0968466e64 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h @@ -12,13 +12,38 @@ #ifndef ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPICSHMHAIN_H_ #define ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPISHMCHAIN_H_ +#include "adios2/common/ADIOSConfig.h" #include "adios2/toolkit/aggregator/mpi/MPIAggregator.h" +#include +#include + namespace adios2 { namespace aggregator { +class Spinlock +{ + /* from + * https://wang-yimu.com/a-tutorial-on-shared-memory-inter-process-communication + */ +public: + std::atomic_flag flag_{ATOMIC_FLAG_INIT}; + void lock() + { + while (!try_lock()) + { + std::this_thread::sleep_for(std::chrono::duration(0.00001)); + } + } + inline bool try_lock() { return !flag_.test_and_set(); } + void unlock() { flag_.clear(); } +}; + +constexpr size_t SHM_BUF_SIZE = 4194304; // 4MB +// we allocate 2x this size + a bit for shared memory segment + /** A one- or two-layer aggregator chain for using Shared memory within a * compute node. * Use MPI split type to group processes within a node into one chain. @@ -85,8 +110,53 @@ class MPIShmChain : public MPIAggregator */ helper::Comm m_AggregatorChainComm; + struct ShmDataBuffer + { + size_t max_size; // max size for buf + size_t actual_size; // size of actual content + // points to data buffer in shared memory + // Warning: This is a different address on every process + char *buf; + }; + + ShmDataBuffer *LockProducerBuffer(); + void UnlockProducerBuffer(); + ShmDataBuffer *LockConsumerBuffer(); + void UnlockConsumerBuffer(); + void ResetBuffers() noexcept; + private: - void HandshakeLinks(); + helper::Comm::Req HandshakeLinks_Start(); + void HandshakeLinks_Complete(helper::Comm::Req &req); + + helper::Comm::Win m_Win; + void CreateShm(); + void DestroyShm(); + + enum class BufferUse + { + None, + A, + B + }; + + struct ShmSegment + { + // -1: none 0-1: which buffer is being filled by producer + BufferUse producerBuffer; + // -1: none 0-1: which buffer is being used by consumer (aggregator) + BufferUse consumerBuffer; + // user facing structs + ShmDataBuffer sdbA; + ShmDataBuffer sdbB; + // locks for individual buffers (sdb and buf) + aggregator::Spinlock lockA; + aggregator::Spinlock lockB; + // the actual data buffers + char bufA[SHM_BUF_SIZE]; + char bufB[SHM_BUF_SIZE]; + }; + ShmSegment *m_Shm; }; } // end namespace aggregator From dcf1e976d8075828b8f618e8a96eddbb17d17720 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 10:08:52 -0400 Subject: [PATCH 09/18] Block producers to get buffer while both buffers are full and unconsumed --- .../toolkit/aggregator/mpi/MPIShmChain.cpp | 75 ++++++++++++++----- .../toolkit/aggregator/mpi/MPIShmChain.h | 9 +-- 2 files changed, 61 insertions(+), 23 deletions(-) diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp index cd3235b7b1..fc64c9117b 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp @@ -229,8 +229,9 @@ void MPIShmChain::CreateShm() m_Comm.Win_shared_query(m_Win, 0, &shmsize, &disp_unit, &ptr); } m_Shm = reinterpret_cast(ptr); - m_Shm->producerBuffer = BufferUse::None; - m_Shm->consumerBuffer = BufferUse::None; + m_Shm->producerBuffer = LastBufferUsed::None; + m_Shm->consumerBuffer = LastBufferUsed::None; + m_Shm->NumBuffersFull = 0; m_Shm->sdbA.buf = nullptr; m_Shm->sdbA.max_size = SHM_BUF_SIZE; m_Shm->sdbB.buf = nullptr; @@ -243,23 +244,60 @@ void MPIShmChain::CreateShm() void MPIShmChain::DestroyShm() { m_Comm.Win_free(m_Win); } +/* + The buffering strategy is the following. + Assumptions: 1. Only one Producer (and one Consumer) is active at a time. + + The first Producer fills buffer A first then B and then is always + alternating, blocking when Consumer is behind (NumBuffersFull == 2). + The next Producer will continue with the alternating pattern where the + previous Producer has finished. + + The Consumer is blocked until there is at least one buffer available. + It takes buffer A at the first call, then it alternates between the two + buffers. + + MPI_Win locking is used to modify the m_Shm variables exclusively (very short + time) C++ atomic locks are used to give long term exclusive access to one + buffer to be filled or consumed. + + The sleeping phases, to wait on the other party to catch up, are outside of + the locking code areas. + + Note: the m_Shm->sdbX.buf pointers must be set on the local process every + time, even tough it is stored on the shared memory segment, because the + address of the segment is different on every process. Failing to set on the + local process causes this pointer pointing to an invalid address (set on + another process). + + Note: the sdbA and sdbB structs are stored on the shared memory segment + because they contain 'actual_size' which is set on the Producer and used by + the Consumer. + +*/ + MPIShmChain::ShmDataBuffer *MPIShmChain::LockProducerBuffer() { MPIShmChain::ShmDataBuffer *sdb = nullptr; + // Sleep until there is a buffer available at all + while (m_Shm->NumBuffersFull == 2) + { + std::this_thread::sleep_for(std::chrono::duration(0.00001)); + } + m_Comm.Win_Lock(helper::Comm::LockType::Exclusive, 0, 0, m_Win); - if (m_Shm->producerBuffer == BufferUse::A) + if (m_Shm->producerBuffer == LastBufferUsed::A) { - m_Shm->producerBuffer = BufferUse::B; + m_Shm->producerBuffer = LastBufferUsed::B; sdb = &m_Shm->sdbB; // point to shm data buffer (in local process memory) sdb->buf = m_Shm->bufB; } - else // (m_Shm->producerBuffer == BufferUse::None || - // m_Shm->producerBuffer == BufferUse::B) + else // None or B { - m_Shm->producerBuffer = BufferUse::A; + m_Shm->producerBuffer = LastBufferUsed::A; sdb = &m_Shm->sdbA; // point to shm data buffer (in local process memory) sdb->buf = m_Shm->bufA; @@ -268,7 +306,7 @@ MPIShmChain::ShmDataBuffer *MPIShmChain::LockProducerBuffer() // We determined we want a specific buffer // Now we need to get a lock on it in case consumer is using it - if (m_Shm->producerBuffer == BufferUse::A) + if (m_Shm->producerBuffer == LastBufferUsed::A) { m_Shm->lockA.lock(); } @@ -282,7 +320,7 @@ MPIShmChain::ShmDataBuffer *MPIShmChain::LockProducerBuffer() void MPIShmChain::UnlockProducerBuffer() { - if (m_Shm->producerBuffer == BufferUse::A) + if (m_Shm->producerBuffer == LastBufferUsed::A) { m_Shm->lockA.unlock(); } @@ -290,14 +328,15 @@ void MPIShmChain::UnlockProducerBuffer() { m_Shm->lockB.unlock(); } + ++m_Shm->NumBuffersFull; } MPIShmChain::ShmDataBuffer *MPIShmChain::LockConsumerBuffer() { MPIShmChain::ShmDataBuffer *sdb = nullptr; - // Sleep until the very first production has started: - while (m_Shm->producerBuffer == BufferUse::None) + // Sleep until there is at least one buffer filled + while (m_Shm->NumBuffersFull < 1) { std::this_thread::sleep_for(std::chrono::duration(0.00001)); } @@ -305,18 +344,17 @@ MPIShmChain::ShmDataBuffer *MPIShmChain::LockConsumerBuffer() // when we successfully lock it m_Comm.Win_Lock(helper::Comm::LockType::Exclusive, 0, 0, m_Win); - if (m_Shm->consumerBuffer == BufferUse::A) + if (m_Shm->consumerBuffer == LastBufferUsed::A) { - m_Shm->consumerBuffer = BufferUse::B; + m_Shm->consumerBuffer = LastBufferUsed::B; sdb = &m_Shm->sdbB; // point to shm data buffer (in local process memory) sdb->buf = m_Shm->bufB; } - else // (m_Shm->consumerBuffer == BufferUse::None || - // m_Shm->consumerBuffer == BufferUse::B) + else // None or B { - m_Shm->consumerBuffer = BufferUse::A; + m_Shm->consumerBuffer = LastBufferUsed::A; sdb = &m_Shm->sdbA; // point to shm data buffer (in local process memory) sdb->buf = m_Shm->bufA; @@ -325,7 +363,7 @@ MPIShmChain::ShmDataBuffer *MPIShmChain::LockConsumerBuffer() // We determined we want a specific buffer // Now we need to get a lock on it in case producer is using it - if (m_Shm->consumerBuffer == BufferUse::A) + if (m_Shm->consumerBuffer == LastBufferUsed::A) { m_Shm->lockA.lock(); } @@ -339,7 +377,7 @@ MPIShmChain::ShmDataBuffer *MPIShmChain::LockConsumerBuffer() void MPIShmChain::UnlockConsumerBuffer() { - if (m_Shm->consumerBuffer == BufferUse::A) + if (m_Shm->consumerBuffer == LastBufferUsed::A) { m_Shm->lockA.unlock(); } @@ -347,6 +385,7 @@ void MPIShmChain::UnlockConsumerBuffer() { m_Shm->lockB.unlock(); } + --m_Shm->NumBuffersFull; } } // end namespace aggregator diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h index 0968466e64..438fdb5bac 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h @@ -133,7 +133,7 @@ class MPIShmChain : public MPIAggregator void CreateShm(); void DestroyShm(); - enum class BufferUse + enum class LastBufferUsed { None, A, @@ -142,10 +142,9 @@ class MPIShmChain : public MPIAggregator struct ShmSegment { - // -1: none 0-1: which buffer is being filled by producer - BufferUse producerBuffer; - // -1: none 0-1: which buffer is being used by consumer (aggregator) - BufferUse consumerBuffer; + LastBufferUsed producerBuffer; + LastBufferUsed consumerBuffer; + unsigned int NumBuffersFull; // user facing structs ShmDataBuffer sdbA; ShmDataBuffer sdbB; From 8ab558604a70b5725d607db708c433fb092c7437 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 10:57:04 -0400 Subject: [PATCH 10/18] fix missing return in a function --- source/adios2/helper/adiosCommDummy.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/source/adios2/helper/adiosCommDummy.cpp b/source/adios2/helper/adiosCommDummy.cpp index 0af8371bc3..081de0cc9d 100644 --- a/source/adios2/helper/adiosCommDummy.cpp +++ b/source/adios2/helper/adiosCommDummy.cpp @@ -328,6 +328,7 @@ int CommImplDummy::Win_shared_query(Comm::Win &win, int rank, size_t *size, int CommImplDummy::Win_free(Comm::Win &win, const std::string &) const { win.Free(); + return 0; } int CommImplDummy::Win_Lock(Comm::LockType lock_type, int rank, int assert, From a763b22d1f20c91dd634eb362bd33a97a1ff6efb Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 11:52:48 -0400 Subject: [PATCH 11/18] FilePOSIX: when opening in chain, non-zero ranks always use Append mode to avoid deleting an file just created by rank 0. Rank 0 always uses non-async open if there are multiple processes opening the file to make sure it is created before others try to open it. --- source/adios2/engine/bp5/BP5Writer.cpp | 15 +++----- source/adios2/engine/bp5/BP5Writer.h | 2 +- .../toolkit/transport/file/FilePOSIX.cpp | 35 ++++++++++++++++--- 3 files changed, 36 insertions(+), 16 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index d82455793d..9591f5b64d 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -431,8 +431,8 @@ void BP5Writer::InitAggregator() m_Parameters.NumSubFiles, m_Comm); m_IAmDraining = m_AggregatorEveroneWrites.m_IsAggregator; m_IAmWritingDataHeader = m_AggregatorEveroneWrites.m_IsAggregator; - m_EveryoneWrites = true; m_IAmWritingData = true; + DataWritingComm = &m_AggregatorEveroneWrites.m_Comm; m_Aggregator = static_cast( &m_AggregatorEveroneWrites); } @@ -455,6 +455,7 @@ void BP5Writer::InitAggregator() m_IAmDraining = m_AggregatorTwoLevelShm.m_IsMasterAggregator; m_IAmWritingData = m_AggregatorTwoLevelShm.m_IsAggregator; m_IAmWritingDataHeader = m_AggregatorTwoLevelShm.m_IsMasterAggregator; + DataWritingComm = &m_AggregatorTwoLevelShm.m_AggregatorChainComm; m_Aggregator = static_cast(&m_AggregatorTwoLevelShm); } @@ -540,19 +541,11 @@ void BP5Writer::InitTransports() } } - if (m_EveryoneWrites) + if (m_IAmWritingData) { m_FileDataManager.OpenFiles(m_SubStreamNames, m_OpenMode, m_IO.m_TransportsParameters, false, - m_Aggregator->m_Comm); - } - else - { - if (m_IAmWritingData) - { - m_FileDataManager.OpenFiles(m_SubStreamNames, m_OpenMode, - m_IO.m_TransportsParameters, false); - } + *DataWritingComm); } if (m_IAmDraining) diff --git a/source/adios2/engine/bp5/BP5Writer.h b/source/adios2/engine/bp5/BP5Writer.h index 9673c4ab87..170c666101 100644 --- a/source/adios2/engine/bp5/BP5Writer.h +++ b/source/adios2/engine/bp5/BP5Writer.h @@ -165,8 +165,8 @@ class BP5Writer : public BP5Engine, public core::Engine aggregator::MPIShmChain m_AggregatorTwoLevelShm; aggregator::MPIChain m_AggregatorEveroneWrites; bool m_IAmDraining = false; - bool m_EveryoneWrites = false; bool m_IAmWritingData = false; + helper::Comm *DataWritingComm; // processes that write the same data file bool m_IAmWritingDataHeader = false; private: diff --git a/source/adios2/toolkit/transport/file/FilePOSIX.cpp b/source/adios2/toolkit/transport/file/FilePOSIX.cpp index f2fabfeb31..1391110faf 100644 --- a/source/adios2/toolkit/transport/file/FilePOSIX.cpp +++ b/source/adios2/toolkit/transport/file/FilePOSIX.cpp @@ -18,6 +18,8 @@ #include // open #include // write, close +#include + /// \cond EXCLUDE_FROM_DOXYGEN #include //std::ios_base::failure /// \endcond @@ -132,6 +134,7 @@ void FilePOSIX::OpenChain(const std::string &name, const Mode openMode, return FD; }; + bool createOnAppend = true; int token = 1; m_Name = name; CheckName(); @@ -140,6 +143,11 @@ void FilePOSIX::OpenChain(const std::string &name, const Mode openMode, { chainComm.Recv(&token, 1, chainComm.Rank() - 1, 0, "Chain token in FilePOSIX::OpenChain"); + if (openMode == Mode::Write) + { + openMode == Mode::Append; + createOnAppend = false; + } } m_OpenMode = openMode; @@ -147,8 +155,9 @@ void FilePOSIX::OpenChain(const std::string &name, const Mode openMode, { case (Mode::Write): - if (async) + if (async && chainComm.Size() == 1) { + // only single process open can do it asynchronously m_IsOpening = true; m_OpenFuture = std::async(std::launch::async, lf_AsyncOpenWrite, name); @@ -167,9 +176,18 @@ void FilePOSIX::OpenChain(const std::string &name, const Mode openMode, case (Mode::Append): ProfilerStart("open"); errno = 0; - // m_FileDescriptor = open(m_Name.c_str(), O_RDWR); - m_FileDescriptor = open(m_Name.c_str(), O_RDWR | O_CREAT, 0777); - lseek(m_FileDescriptor, 0, SEEK_END); + + if (createOnAppend) + { + m_FileDescriptor = open(m_Name.c_str(), O_RDWR | O_CREAT, 0777); + lseek(m_FileDescriptor, 0, SEEK_END); + } + else + { + m_FileDescriptor = open(m_Name.c_str(), O_RDWR); + lseek(m_FileDescriptor, 0, SEEK_SET); + } + m_Errno = errno; ProfilerStop("open"); break; @@ -243,6 +261,15 @@ void FilePOSIX::Write(const char *buffer, size_t size, size_t start) ", in call to POSIX lseek" + SysErrMsg()); } } + else + { + const auto pos = lseek(m_FileDescriptor, 0, SEEK_CUR); + start = static_cast(pos); + } + std::cout << " Write to " << m_Name << " size = " << size + << " pos = " << start << " buf = [" << (int)buffer[0] + << (int)buffer[1] << "..." << (int)buffer[size - 2] + << (int)buffer[size - 1] << "]" << std::endl; if (size > DefaultMaxFileBatchSize) { From ae65119547064c07ccf49a4bfd226838b9cd956a Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 14:03:52 -0400 Subject: [PATCH 12/18] Add chain open to FStream transport. --- .../toolkit/transport/file/FileFStream.cpp | 90 +++++++++++++++++++ .../toolkit/transport/file/FileFStream.h | 4 + .../toolkit/transport/file/FilePOSIX.cpp | 5 +- 3 files changed, 95 insertions(+), 4 deletions(-) diff --git a/source/adios2/toolkit/transport/file/FileFStream.cpp b/source/adios2/toolkit/transport/file/FileFStream.cpp index 50e7169567..440563e594 100644 --- a/source/adios2/toolkit/transport/file/FileFStream.cpp +++ b/source/adios2/toolkit/transport/file/FileFStream.cpp @@ -101,6 +101,96 @@ void FileFStream::Open(const std::string &name, const Mode openMode, } } +void FileFStream::OpenChain(const std::string &name, const Mode openMode, + const helper::Comm &chainComm, const bool async) +{ + auto lf_AsyncOpenWrite = [&](const std::string &name) -> void { + ProfilerStart("open"); + m_FileStream.open(name, std::fstream::out | std::fstream::binary | + std::fstream::trunc); + ProfilerStop("open"); + }; + + bool createOnAppend = true; + int token = 1; + m_Name = name; + CheckName(); + + if (chainComm.Rank() > 0) + { + chainComm.Recv(&token, 1, chainComm.Rank() - 1, 0, + "Chain token in FileFStream::OpenChain"); + if (openMode == Mode::Write) + { + openMode == Mode::Append; + createOnAppend = false; + } + } + + m_OpenMode = openMode; + switch (m_OpenMode) + { + case (Mode::Write): + if (async && chainComm.Size() == 1) + { + m_IsOpening = true; + m_OpenFuture = + std::async(std::launch::async, lf_AsyncOpenWrite, name); + } + else + { + ProfilerStart("open"); + m_FileStream.open(name, std::fstream::out | std::fstream::binary | + std::fstream::trunc); + ProfilerStop("open"); + } + break; + + case (Mode::Append): + ProfilerStart("open"); + + if (createOnAppend) + { + m_FileStream.open(name, std::fstream::in | std::fstream::out | + std::fstream::binary); + m_FileStream.seekp(0, std::ios_base::end); + } + else + { + m_FileStream.open(name, std::fstream::in | std::fstream::out | + std::fstream::binary); + m_FileStream.seekp(0, std::ios_base::beg); + } + + ProfilerStop("open"); + break; + + case (Mode::Read): + ProfilerStart("open"); + m_FileStream.open(name, std::fstream::in | std::fstream::binary); + ProfilerStop("open"); + break; + + default: + CheckFile("unknown open mode for file " + m_Name + + ", in call to stream open"); + } + + if (!m_IsOpening) + { + CheckFile( + "couldn't open file " + m_Name + + ", check permissions or path existence, in call to fstream open"); + m_IsOpen = true; + } + + if (chainComm.Rank() < chainComm.Size() - 1) + { + chainComm.Isend(&token, 1, chainComm.Rank() + 1, 0, + "Sending Chain token in FileFStream::OpenChain"); + } +} + void FileFStream::SetBuffer(char *buffer, size_t size) { if (!buffer && size != 0) diff --git a/source/adios2/toolkit/transport/file/FileFStream.h b/source/adios2/toolkit/transport/file/FileFStream.h index 1c44e9bd9b..d066a9d2d1 100644 --- a/source/adios2/toolkit/transport/file/FileFStream.h +++ b/source/adios2/toolkit/transport/file/FileFStream.h @@ -35,6 +35,10 @@ class FileFStream : public Transport void Open(const std::string &name, const Mode openMode, const bool async = false) final; + void OpenChain(const std::string &name, const Mode openMode, + const helper::Comm &chainComm, + const bool async = false) final; + void SetBuffer(char *buffer, size_t size) final; void Write(const char *buffer, size_t size, size_t start = MaxSizeT) final; diff --git a/source/adios2/toolkit/transport/file/FilePOSIX.cpp b/source/adios2/toolkit/transport/file/FilePOSIX.cpp index 1391110faf..a13c3dc827 100644 --- a/source/adios2/toolkit/transport/file/FilePOSIX.cpp +++ b/source/adios2/toolkit/transport/file/FilePOSIX.cpp @@ -184,6 +184,7 @@ void FilePOSIX::OpenChain(const std::string &name, const Mode openMode, } else { + /* This case runs on rank > 0 when called with Write mode */ m_FileDescriptor = open(m_Name.c_str(), O_RDWR); lseek(m_FileDescriptor, 0, SEEK_SET); } @@ -266,10 +267,6 @@ void FilePOSIX::Write(const char *buffer, size_t size, size_t start) const auto pos = lseek(m_FileDescriptor, 0, SEEK_CUR); start = static_cast(pos); } - std::cout << " Write to " << m_Name << " size = " << size - << " pos = " << start << " buf = [" << (int)buffer[0] - << (int)buffer[1] << "..." << (int)buffer[size - 2] - << (int)buffer[size - 1] << "]" << std::endl; if (size > DefaultMaxFileBatchSize) { From 2dafa3fcb13325e16bafa29221cec3be5115bbd7 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 14:04:17 -0400 Subject: [PATCH 13/18] Remove debug prints --- source/adios2/engine/bp5/BP5Writer.cpp | 4 ++-- .../engine/bp5/BP5Writer_TwoLevelShm.cpp | 20 +++++++++++-------- .../toolkit/aggregator/mpi/MPIShmChain.cpp | 4 ++-- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index 9591f5b64d..bb0da95be4 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -442,7 +442,7 @@ void BP5Writer::InitAggregator() m_AggregatorTwoLevelShm.Init(m_Parameters.NumAggregators, m_Parameters.NumSubFiles, m_Comm); - std::cout << "Rank " << m_RankMPI << " aggr? " + /*std::cout << "Rank " << m_RankMPI << " aggr? " << m_AggregatorTwoLevelShm.m_IsAggregator << " master? " << m_AggregatorTwoLevelShm.m_IsMasterAggregator << " aggr size = " << m_AggregatorTwoLevelShm.m_Size @@ -450,7 +450,7 @@ void BP5Writer::InitAggregator() << " subfile = " << m_AggregatorTwoLevelShm.m_SubStreamIndex << " type = " << m_Parameters.AggregationType - << std::endl; + << std::endl;*/ m_IAmDraining = m_AggregatorTwoLevelShm.m_IsMasterAggregator; m_IAmWritingData = m_AggregatorTwoLevelShm.m_IsAggregator; diff --git a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp index f7c12e5fac..2aaced4374 100644 --- a/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp +++ b/source/adios2/engine/bp5/BP5Writer_TwoLevelShm.cpp @@ -98,10 +98,12 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) a->m_AggregatorChainComm.Isend( &nextWriterPos, 1, 0, 0, "Chain token in BP5Writer::WriteData"); } - std::cout << "Rank " << m_Comm.Rank() + + /*std::cout << "Rank " << m_Comm.Rank() << " aggregator start writing step " << m_WriterStep << " to subfile " << a->m_SubStreamIndex << " at pos " - << m_DataPos << " totalsize " << myTotalSize << std::endl; + << m_DataPos << " totalsize " << myTotalSize << std::endl;*/ + // Send token to first non-aggregator to start filling shm // Also informs next process its starting offset (for correct metadata) if (a->m_Comm.Size() > 1) @@ -136,9 +138,10 @@ void BP5Writer::WriteData_TwoLevelShm(format::BufferV *Data) // they also receive their starting offset this way a->m_Comm.Recv(&m_StartDataPos, 1, a->m_Comm.Rank() - 1, 0, "Shm token in BP5Writer::WriteData_TwoLevelShm"); - std::cout << "Rank " << m_Comm.Rank() + + /*std::cout << "Rank " << m_Comm.Rank() << " non-aggregator recv token to fill shm = " - << m_StartDataPos << std::endl; + << m_StartDataPos << std::endl;*/ SendDataToAggregator(DataVec, Data->Size()); @@ -233,14 +236,15 @@ void BP5Writer::SendDataToAggregator(format::BufferV::BufferV_iovec DataVec, } } sent += b->actual_size; - std::cout << "Rank " << m_Comm.Rank() + + /*std::cout << "Rank " << m_Comm.Rank() << " filled shm, data_size = " << b->actual_size << " block = " << block << " temp offset = " << temp_offset << " sent = " << sent << " buf = " << static_cast(b->buf) << " = [" << (int)b->buf[0] << (int)b->buf[1] << "..." << (int)b->buf[b->actual_size - 2] - << (int)b->buf[b->actual_size - 1] << "]" << std::endl; + << (int)b->buf[b->actual_size - 1] << "]" << std::endl;*/ a->UnlockProducerBuffer(); } @@ -257,13 +261,13 @@ void BP5Writer::WriteOthersData(size_t TotalSize) // potentially blocking call waiting on some non-aggr process aggregator::MPIShmChain::ShmDataBuffer *b = a->LockConsumerBuffer(); - std::cout << "Rank " << m_Comm.Rank() + /*std::cout << "Rank " << m_Comm.Rank() << " write from shm, data_size = " << b->actual_size << " total so far = " << wrote << " buf = " << static_cast(b->buf) << " = [" << (int)b->buf[0] << (int)b->buf[1] << "..." << (int)b->buf[b->actual_size - 2] - << (int)b->buf[b->actual_size - 1] << "]" << std::endl; + << (int)b->buf[b->actual_size - 1] << "]" << std::endl;*/ // b->actual_size: how much we need to write m_FileDataManager.WriteFiles(b->buf, b->actual_size); diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp index fc64c9117b..6927544918 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp @@ -237,9 +237,9 @@ void MPIShmChain::CreateShm() m_Shm->sdbB.buf = nullptr; m_Shm->sdbB.max_size = SHM_BUF_SIZE; - std::cout << "Rank " << m_Rank << " shm = " << ptr + /*std::cout << "Rank " << m_Rank << " shm = " << ptr << " bufA = " << static_cast(m_Shm->bufA) - << " bufB = " << static_cast(m_Shm->bufB) << std::endl; + << " bufB = " << static_cast(m_Shm->bufB) << std::endl;*/ } void MPIShmChain::DestroyShm() { m_Comm.Win_free(m_Win); } From f3c5dd52fa5611c088b6e12f438015ab3f1f08ca Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 14:30:07 -0400 Subject: [PATCH 14/18] BP5: Set default aggregation to TwoLevelShm, 1 aggregator per node, 1 file per aggregator, 4096 page size --- source/adios2/engine/bp5/BP5Engine.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Engine.h b/source/adios2/engine/bp5/BP5Engine.h index 256beb5eab..6bc187675d 100644 --- a/source/adios2/engine/bp5/BP5Engine.h +++ b/source/adios2/engine/bp5/BP5Engine.h @@ -110,11 +110,11 @@ class BP5Engine MACRO(NodeLocal, Bool, bool, false) \ MACRO(verbose, Int, int, 0) \ MACRO(CollectiveMetadata, Bool, bool, true) \ - MACRO(NumAggregators, UInt, unsigned int, 999999) \ + MACRO(NumAggregators, UInt, unsigned int, 0) \ MACRO(NumSubFiles, UInt, unsigned int, 999999) \ - MACRO(FileSystemPageSize, UInt, unsigned int, 65536) \ + MACRO(FileSystemPageSize, UInt, unsigned int, 4096) \ MACRO(AggregationType, AggregationType, int, \ - (int)AggregationType::EveryoneWrites) \ + (int)AggregationType::TwoLevelShm) \ MACRO(AsyncTasks, Bool, bool, true) \ MACRO(GrowthFactor, Float, float, DefaultBufferGrowthFactor) \ MACRO(InitialBufferSize, SizeBytes, size_t, DefaultInitialBufferSize) \ From a5a89f19eead12dee296970d257d1999cae86fc1 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 15:42:12 -0400 Subject: [PATCH 15/18] fix compiler warnings --- source/adios2/engine/bp5/BP5Writer.cpp | 2 +- source/adios2/helper/adiosCommDummy.cpp | 4 ++-- source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp | 2 +- source/adios2/toolkit/aggregator/mpi/MPIShmChain.h | 10 +++++++--- source/adios2/toolkit/transport/Transport.h | 2 +- source/adios2/toolkit/transport/file/FileFStream.cpp | 2 +- source/adios2/toolkit/transport/file/FileFStream.h | 2 +- source/adios2/toolkit/transport/file/FilePOSIX.cpp | 2 +- source/adios2/toolkit/transport/file/FilePOSIX.h | 2 +- 9 files changed, 16 insertions(+), 12 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Writer.cpp b/source/adios2/engine/bp5/BP5Writer.cpp index 8d0d8c6551..b4e5b5e19c 100644 --- a/source/adios2/engine/bp5/BP5Writer.cpp +++ b/source/adios2/engine/bp5/BP5Writer.cpp @@ -425,7 +425,7 @@ void BP5Writer::InitParameters() if (m_Parameters.FileSystemPageSize == 0) { - m_Parameters.FileSystemPageSize = 65536; + m_Parameters.FileSystemPageSize = 4096; } if (m_Parameters.FileSystemPageSize > 67108864) { diff --git a/source/adios2/helper/adiosCommDummy.cpp b/source/adios2/helper/adiosCommDummy.cpp index 081de0cc9d..7c4ae8fd00 100644 --- a/source/adios2/helper/adiosCommDummy.cpp +++ b/source/adios2/helper/adiosCommDummy.cpp @@ -311,7 +311,7 @@ Comm::Win CommImplDummy::Win_allocate_shared(size_t size, int disp_unit, const std::string &) const { auto win = std::unique_ptr(new CommWinImplDummy()); - baseptr = nullptr; + // TODO: How do you set the out pointer to NULL? baseptr = nullptr; return MakeWin(std::move(win)); } @@ -321,7 +321,7 @@ int CommImplDummy::Win_shared_query(Comm::Win &win, int rank, size_t *size, { *size = 0; *disp_unit = 1; - baseptr = nullptr; + // TODO: How do you set the out pointer to NULL? baseptr = nullptr; return 0; } diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp index 6927544918..6f1fb04ee3 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.cpp @@ -151,7 +151,7 @@ void MPIShmChain::Init(const size_t numAggregators, const size_t subStreams, /* Create the communicator to connect aggregators writing to the same * substream */ - color = m_SubStreamIndex; + color = static_cast(m_SubStreamIndex); m_AggregatorChainComm = m_AllAggregatorsComm.Split( color, 0, "creating chains of aggregators at Open"); diff --git a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h index 438fdb5bac..35dd9fdbde 100644 --- a/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h +++ b/source/adios2/toolkit/aggregator/mpi/MPIShmChain.h @@ -9,12 +9,13 @@ * */ -#ifndef ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPICSHMHAIN_H_ +#ifndef ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPISHMCHAIN_H_ #define ADIOS2_TOOLKIT_AGGREGATOR_MPI_MPISHMCHAIN_H_ #include "adios2/common/ADIOSConfig.h" #include "adios2/toolkit/aggregator/mpi/MPIAggregator.h" +#include #include #include @@ -29,7 +30,7 @@ class Spinlock * https://wang-yimu.com/a-tutorial-on-shared-memory-inter-process-communication */ public: - std::atomic_flag flag_{ATOMIC_FLAG_INIT}; + Spinlock() { flag_.clear(); } void lock() { while (!try_lock()) @@ -37,8 +38,11 @@ class Spinlock std::this_thread::sleep_for(std::chrono::duration(0.00001)); } } - inline bool try_lock() { return !flag_.test_and_set(); } void unlock() { flag_.clear(); } + +private: + inline bool try_lock() { return !flag_.test_and_set(); } + std::atomic_flag flag_; //{ATOMIC_FLAG_INIT}; }; constexpr size_t SHM_BUF_SIZE = 4194304; // 4MB diff --git a/source/adios2/toolkit/transport/Transport.h b/source/adios2/toolkit/transport/Transport.h index 9e0a87176c..688febe86b 100644 --- a/source/adios2/toolkit/transport/Transport.h +++ b/source/adios2/toolkit/transport/Transport.h @@ -75,7 +75,7 @@ class Transport * @param chainComm * @param async */ - virtual void OpenChain(const std::string &name, const Mode openMode, + virtual void OpenChain(const std::string &name, Mode openMode, const helper::Comm &chainComm, const bool async = false); diff --git a/source/adios2/toolkit/transport/file/FileFStream.cpp b/source/adios2/toolkit/transport/file/FileFStream.cpp index 440563e594..3af5389c5a 100644 --- a/source/adios2/toolkit/transport/file/FileFStream.cpp +++ b/source/adios2/toolkit/transport/file/FileFStream.cpp @@ -101,7 +101,7 @@ void FileFStream::Open(const std::string &name, const Mode openMode, } } -void FileFStream::OpenChain(const std::string &name, const Mode openMode, +void FileFStream::OpenChain(const std::string &name, Mode openMode, const helper::Comm &chainComm, const bool async) { auto lf_AsyncOpenWrite = [&](const std::string &name) -> void { diff --git a/source/adios2/toolkit/transport/file/FileFStream.h b/source/adios2/toolkit/transport/file/FileFStream.h index d066a9d2d1..9e47c4b2e8 100644 --- a/source/adios2/toolkit/transport/file/FileFStream.h +++ b/source/adios2/toolkit/transport/file/FileFStream.h @@ -35,7 +35,7 @@ class FileFStream : public Transport void Open(const std::string &name, const Mode openMode, const bool async = false) final; - void OpenChain(const std::string &name, const Mode openMode, + void OpenChain(const std::string &name, Mode openMode, const helper::Comm &chainComm, const bool async = false) final; diff --git a/source/adios2/toolkit/transport/file/FilePOSIX.cpp b/source/adios2/toolkit/transport/file/FilePOSIX.cpp index a13c3dc827..35b05ccbf7 100644 --- a/source/adios2/toolkit/transport/file/FilePOSIX.cpp +++ b/source/adios2/toolkit/transport/file/FilePOSIX.cpp @@ -122,7 +122,7 @@ void FilePOSIX::Open(const std::string &name, const Mode openMode, } } -void FilePOSIX::OpenChain(const std::string &name, const Mode openMode, +void FilePOSIX::OpenChain(const std::string &name, Mode openMode, const helper::Comm &chainComm, const bool async) { auto lf_AsyncOpenWrite = [&](const std::string &name) -> int { diff --git a/source/adios2/toolkit/transport/file/FilePOSIX.h b/source/adios2/toolkit/transport/file/FilePOSIX.h index 8d9b0300d2..5e6cb7f94a 100644 --- a/source/adios2/toolkit/transport/file/FilePOSIX.h +++ b/source/adios2/toolkit/transport/file/FilePOSIX.h @@ -37,7 +37,7 @@ class FilePOSIX : public Transport void Open(const std::string &name, const Mode openMode, const bool async = false) final; - void OpenChain(const std::string &name, const Mode openMode, + void OpenChain(const std::string &name, Mode openMode, const helper::Comm &chainComm, const bool async = false) final; From b6d6dbd8e349ec0f221ce7ba40085b7e705ab266 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 16:06:46 -0400 Subject: [PATCH 16/18] fix warnings --- source/adios2/toolkit/transport/file/FileFStream.cpp | 2 +- source/adios2/toolkit/transport/file/FilePOSIX.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/adios2/toolkit/transport/file/FileFStream.cpp b/source/adios2/toolkit/transport/file/FileFStream.cpp index 3af5389c5a..c71e5fdfe7 100644 --- a/source/adios2/toolkit/transport/file/FileFStream.cpp +++ b/source/adios2/toolkit/transport/file/FileFStream.cpp @@ -122,7 +122,7 @@ void FileFStream::OpenChain(const std::string &name, Mode openMode, "Chain token in FileFStream::OpenChain"); if (openMode == Mode::Write) { - openMode == Mode::Append; + openMode = Mode::Append; createOnAppend = false; } } diff --git a/source/adios2/toolkit/transport/file/FilePOSIX.cpp b/source/adios2/toolkit/transport/file/FilePOSIX.cpp index 35b05ccbf7..7ec300be4a 100644 --- a/source/adios2/toolkit/transport/file/FilePOSIX.cpp +++ b/source/adios2/toolkit/transport/file/FilePOSIX.cpp @@ -145,7 +145,7 @@ void FilePOSIX::OpenChain(const std::string &name, Mode openMode, "Chain token in FilePOSIX::OpenChain"); if (openMode == Mode::Write) { - openMode == Mode::Append; + openMode = Mode::Append; createOnAppend = false; } } From f88c71ffc6fb9338c7c025672cd809f52f87a122 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 16:25:26 -0400 Subject: [PATCH 17/18] remove superflous and wrong line in BP5Engine::ParseParameters --- source/adios2/engine/bp5/BP5Engine.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/source/adios2/engine/bp5/BP5Engine.cpp b/source/adios2/engine/bp5/BP5Engine.cpp index 07ae493273..3be352b204 100644 --- a/source/adios2/engine/bp5/BP5Engine.cpp +++ b/source/adios2/engine/bp5/BP5Engine.cpp @@ -192,8 +192,6 @@ void BP5Engine::ParseParams(IO &io, struct BP5Params &Params) std::string value = itKey->second; parameter = helper::StringToByteUnits( value, "for Parameter key=" + key + "in call to Open"); - parameter = - helper::StringTo(value, " in Parameter key=" + key); } }; From 585a56cec11799870663a37afe363042f916a866 Mon Sep 17 00:00:00 2001 From: Norbert Podhorszki Date: Thu, 15 Jul 2021 17:53:09 -0400 Subject: [PATCH 18/18] set default NumAggregators to be N-to-N to pass ctest. Does not fix the underlying problem with test Staging.5x3.BP5 producing corrupt data. --- source/adios2/engine/bp5/BP5Engine.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/adios2/engine/bp5/BP5Engine.h b/source/adios2/engine/bp5/BP5Engine.h index 6bc187675d..7841caf5cf 100644 --- a/source/adios2/engine/bp5/BP5Engine.h +++ b/source/adios2/engine/bp5/BP5Engine.h @@ -110,7 +110,7 @@ class BP5Engine MACRO(NodeLocal, Bool, bool, false) \ MACRO(verbose, Int, int, 0) \ MACRO(CollectiveMetadata, Bool, bool, true) \ - MACRO(NumAggregators, UInt, unsigned int, 0) \ + MACRO(NumAggregators, UInt, unsigned int, 999999) \ MACRO(NumSubFiles, UInt, unsigned int, 999999) \ MACRO(FileSystemPageSize, UInt, unsigned int, 4096) \ MACRO(AggregationType, AggregationType, int, \