diff options
author | maxim-yurchuk <maxim-yurchuk@yandex-team.com> | 2024-10-15 12:45:36 +0300 |
---|---|---|
committer | maxim-yurchuk <maxim-yurchuk@yandex-team.com> | 2024-10-15 13:02:02 +0300 |
commit | 2ff5baf34b9dea61580028eb4c4afde7601232a7 (patch) | |
tree | 92b551d6b3afbe7f32517f1bf7f76f5191d3cdc3 | |
parent | 390e5cb4286ef21d5729f0bd330b1d9eb2f80db0 (diff) | |
download | ydb-2ff5baf34b9dea61580028eb4c4afde7601232a7.tar.gz |
Add yt/yt/ytlib into YDB GH sync
See
<HIDDEN_URL>
commit_hash:feb8eec1b66b043af8f4c1eca638b4e081d9334e
-rw-r--r-- | yt/yt/ytlib/chunk_client/public.h | 284 | ||||
-rw-r--r-- | yt/yt/ytlib/misc/public.h | 51 | ||||
-rw-r--r-- | yt/yt/ytlib/node_tracker_client/public.h | 89 | ||||
-rw-r--r-- | yt/yt/ytlib/object_client/public.h | 61 | ||||
-rw-r--r-- | yt/yt/ytlib/table_client/public.h | 170 | ||||
-rw-r--r-- | yt/yt/ytlib/table_client/table_upload_options.h | 87 |
6 files changed, 742 insertions, 0 deletions
diff --git a/yt/yt/ytlib/chunk_client/public.h b/yt/yt/ytlib/chunk_client/public.h new file mode 100644 index 0000000000..0cb2ae86d4 --- /dev/null +++ b/yt/yt/ytlib/chunk_client/public.h @@ -0,0 +1,284 @@ +#pragma once + +#include <yt/yt/ytlib/misc/public.h> + +#include <yt/yt/ytlib/object_client/public.h> + +#include <yt/yt/ytlib/node_tracker_client/public.h> + +#include <yt/yt/client/chunk_client/public.h> + +#include <yt/yt/client/table_client/public.h> + +#include <yt/yt/core/concurrency/async_semaphore.h> + +#include <library/cpp/yt/small_containers/compact_vector.h> + +#include <optional> + +namespace NYT::NChunkClient { + +//////////////////////////////////////////////////////////////////////////////// + +namespace NProto { + +//////////////////////////////////////////////////////////////////////////////// + +class TReqFetch; + +class TReqExportChunks; +class TRspExportChunks; + +class TReqImportChunks; +class TRspImportChunks; + +class TReqExecuteBatch; +class TRspExecuteBatch; + +class TDataSource; +class TDataSourceDirectoryExt; + +class TDataSink; +class TDataSinkDirectoryExt; + +class TReqGetChunkMeta; + +class TAllyReplicasInfo; +class TChunkReplicaAnnouncement; +class TChunkReplicaAnnouncementRequest; + +//////////////////////////////////////////////////////////////////////////////// + +} // namespace NProto + +//////////////////////////////////////////////////////////////////////////////// + +using NTableClient::TLoadContext; +using NTableClient::TSaveContext; +using NTableClient::TPersistenceContext; + +constexpr int MaxMasterChunkMetaExtensions = 6; + +struct TBlock; + +using TMediumId = NObjectClient::TObjectId; + +using TReadSessionId = NObjectClient::TObjectId; + +struct TSessionId; + +constexpr NRpc::TRealmId ProxyingDataNodeServiceRealmId = TGuid(0xd452d72f, 0x3142caa3); + +constexpr int DefaultPartIndex = -1; + +//! Estimated memory overhead per chunk reader. +constexpr i64 ChunkReaderMemorySize = 16_KB; + +constexpr int MaxMediumPriority = 10; + +constexpr i64 DefaultMaxBlockSize = 16_MB; +constexpr int MaxInputChunkReplicaCount = 16; + +//! Represents an offset inside a chunk. +using TBlockOffset = i64; + +//! A |(chunkId, blockIndex)| pair. +struct TBlockId; + +using TConsistentReplicaPlacementHash = ui64; +constexpr TConsistentReplicaPlacementHash NullConsistentReplicaPlacementHash = 0; + +//! All chunks are uniformly divided into |ChunkShardCount| shards. +// BEWARE: Changing this value requires reign promotion since rolling update +// is not possible. +constexpr int ChunkShardCount = 60; +static_assert(ChunkShardCount < std::numeric_limits<i8>::max(), "ChunkShardCount must fit into i8"); + +//! Typical chunk location count per data node. +constexpr int TypicalChunkLocationCount = 20; + +struct TAllyReplicasInfo; + +constexpr int WholeBlockFragmentRequestLength = -1; + +DEFINE_BIT_ENUM(EBlockType, + ((None) (0x0000)) + //! This basically comprises any block regardless of its semantics (data or some system block). + ((CompressedData) (0x0001)) + //! Uncompressed data block. + ((UncompressedData) (0x0002)) + //! Hash table chunk index system block. + ((HashTableChunkIndex) (0x0004)) + //! Xor filter system block. + ((XorFilter) (0x0008)) + //! Blocks used by chunk fragment reader cache. + ((ChunkFragmentsData) (0x0010)) +); + +DEFINE_ENUM(EChunkType, + ((Unknown) (0)) + ((File) (1)) + ((Table) (2)) + ((Journal) (3)) + ((Hunk) (4)) +); + +//! Values must be contiguous. +DEFINE_ENUM(ESessionType, + ((User) (0)) + ((Replication) (1)) + ((Repair) (2)) +); + +DEFINE_ENUM(EUpdateMode, + ((None) (0)) + ((Append) (1)) + ((Overwrite) (2)) +); + +DEFINE_BIT_ENUM(EChunkFeatures, + ((None) (0x0000)) + ((DescendingSortOrder) (0x0001)) + ((StripedErasure) (0x0002)) + ((IndexedBlockFormat) (0x0004)) + ((SlimBlockFormat) (0x0008)) + ((UnversionedHunks) (0x0010)) + ((CompressedHunkValues) (0x0020)) + ((NoColumnMetaInChunkMeta) (0x0040)) +); + +DEFINE_ENUM(EChunkClientFeature, + // COMPAT(akozhikhov). + ((AllBlocksIndex) (0)) +); + +DEFINE_ENUM(EChunkMergerMode, + ((None) (0)) + ((Shallow) (1)) + ((Deep) (2)) + ((Auto) (3)) +); + +DEFINE_ENUM(EChunkListContentType, + ((Main) (0)) + ((Hunk) (1)) +); + +//////////////////////////////////////////////////////////////////////////////// + +DECLARE_REFCOUNTED_CLASS(TRemoteReaderOptions) +DECLARE_REFCOUNTED_CLASS(TDispatcherConfig) +DECLARE_REFCOUNTED_CLASS(TDispatcherDynamicConfig) +DECLARE_REFCOUNTED_CLASS(TMultiChunkWriterOptions) +DECLARE_REFCOUNTED_CLASS(TMultiChunkReaderOptions) +DECLARE_REFCOUNTED_CLASS(TRemoteWriterOptions) +DECLARE_REFCOUNTED_CLASS(TMetaAggregatingWriterOptions) +DECLARE_REFCOUNTED_CLASS(TBlockCacheConfig) +DECLARE_REFCOUNTED_CLASS(TBlockCacheDynamicConfig) +DECLARE_REFCOUNTED_CLASS(TClientChunkMetaCacheConfig) +DECLARE_REFCOUNTED_CLASS(TChunkScraperConfig) +DECLARE_REFCOUNTED_CLASS(TChunkTeleporterConfig) +DECLARE_REFCOUNTED_CLASS(TMediumDirectorySynchronizerConfig) +DECLARE_REFCOUNTED_CLASS(TChunkReplicaCacheConfig) + +DECLARE_REFCOUNTED_STRUCT(IFetcherChunkScraper) + +DECLARE_REFCOUNTED_CLASS(TEncodingWriter) +DECLARE_REFCOUNTED_CLASS(TEncodingChunkWriter) +DECLARE_REFCOUNTED_CLASS(TBlockFetcher) +DECLARE_REFCOUNTED_CLASS(TSequentialBlockFetcher) + +DECLARE_REFCOUNTED_STRUCT(IChunkReader) +DECLARE_REFCOUNTED_STRUCT(IChunkFragmentReader) +DECLARE_REFCOUNTED_STRUCT(IChunkReaderAllowingRepair) + +DECLARE_REFCOUNTED_STRUCT(IReaderBase) +DECLARE_REFCOUNTED_STRUCT(IReaderFactory) + +DECLARE_REFCOUNTED_STRUCT(IMultiReaderManager) + +DECLARE_REFCOUNTED_CLASS(TTrafficMeter) + +DECLARE_REFCOUNTED_STRUCT(IChunkWriterBase) +DECLARE_REFCOUNTED_STRUCT(IMultiChunkWriter) +DECLARE_REFCOUNTED_STRUCT(IChunkWriter) +DECLARE_REFCOUNTED_STRUCT(IMetaAggregatingWriter) + +DECLARE_REFCOUNTED_STRUCT(IBlockCache) +DECLARE_REFCOUNTED_STRUCT(IClientBlockCache) + +DECLARE_REFCOUNTED_CLASS(TMemoryWriter) + +DECLARE_REFCOUNTED_CLASS(TInputChunk) +DECLARE_REFCOUNTED_CLASS(TInputChunkSlice) +DECLARE_REFCOUNTED_CLASS(TWeightedInputChunk) + +DECLARE_REFCOUNTED_STRUCT(TLegacyDataSlice) + +DECLARE_REFCOUNTED_CLASS(TDataSourceDirectory) +DECLARE_REFCOUNTED_CLASS(TDataSinkDirectory) + +DECLARE_REFCOUNTED_CLASS(TChunkScraper) +DECLARE_REFCOUNTED_CLASS(TScraperTask) +DECLARE_REFCOUNTED_CLASS(TThrottlerManager) +DECLARE_REFCOUNTED_CLASS(TChunkTeleporter) +DECLARE_REFCOUNTED_CLASS(TMediumDirectory) +DECLARE_REFCOUNTED_CLASS(TMediumDirectorySynchronizer) + +DECLARE_REFCOUNTED_CLASS(TChunkMetaFetcher) + +DECLARE_REFCOUNTED_CLASS(TMasterChunkSpecFetcher) +DECLARE_REFCOUNTED_CLASS(TTabletChunkSpecFetcher) + +DECLARE_REFCOUNTED_STRUCT(TChunkReaderStatistics) + +DECLARE_REFCOUNTED_STRUCT(IReaderMemoryManager) +DECLARE_REFCOUNTED_CLASS(TChunkReaderMemoryManager) + +DECLARE_REFCOUNTED_STRUCT(IChunkReplicaCache) + +DECLARE_REFCOUNTED_STRUCT(TChunkReaderHost) + +struct TChunkReaderMemoryManagerOptions; + +struct TUserObject; + +using TRefCountedChunkMeta = TRefCountedProto<NChunkClient::NProto::TChunkMeta>; +DECLARE_REFCOUNTED_TYPE(TRefCountedChunkMeta) + +DECLARE_REFCOUNTED_CLASS(TDeferredChunkMeta) + +DECLARE_REFCOUNTED_CLASS(TMemoryTrackedDeferredChunkMeta) + +// NB: TRefCountedBlocksExt needs weak pointers support. +using TRefCountedBlocksExt = TRefCountedProto<NChunkClient::NProto::TBlocksExt>; +DECLARE_REFCOUNTED_TYPE(TRefCountedBlocksExt) + +using TRefCountedMiscExt = TRefCountedProto<NChunkClient::NProto::TMiscExt>; +DECLARE_REFCOUNTED_TYPE(TRefCountedMiscExt) + +using TPlacementId = TGuid; + +struct TDataSliceDescriptor; + +struct TInterruptDescriptor; + +class TCodecStatistics; + +struct TClientChunkReadOptions; + +using TDataCenterName = std::optional<std::string>; + +DECLARE_REFCOUNTED_CLASS(TMemoryUsageGuard) + +DECLARE_REFCOUNTED_CLASS(TChunkReaderMemoryManagerHolder) + +DECLARE_REFCOUNTED_STRUCT(IMultiReaderMemoryManager) +DECLARE_REFCOUNTED_STRUCT(IReaderMemoryManagerHost) + +DECLARE_REFCOUNTED_STRUCT(ICachedChunkMeta) +DECLARE_REFCOUNTED_STRUCT(IClientChunkMetaCache) + +//////////////////////////////////////////////////////////////////////////////// + +} // namespace NYT::NChunkClient diff --git a/yt/yt/ytlib/misc/public.h b/yt/yt/ytlib/misc/public.h new file mode 100644 index 0000000000..a25d6f9aeb --- /dev/null +++ b/yt/yt/ytlib/misc/public.h @@ -0,0 +1,51 @@ +#pragma once + +#include <yt/yt/client/misc/public.h> + +namespace NYT { + +//////////////////////////////////////////////////////////////////////////////// + +DECLARE_REFCOUNTED_STRUCT(INodeMemoryTracker) + +//////////////////////////////////////////////////////////////////////////////// + +DEFINE_ENUM(EMemoryCategory, + ((Footprint) (0)) + ((BlockCache) (1)) + ((ChunkMeta) (2)) + ((ChunkBlockMeta) (10)) + ((ChunkBlocksExt) (16)) + ((ChunkJournalIndex) (20)) + ((Rpc) (21)) + ((UserJobs) (3)) + ((TabletStatic) (4)) + ((TabletDynamic) (5)) + // COMPAT(babenko): drop + ((BlobSession) (6)) + ((PendingDiskRead) (22)) + ((PendingDiskWrite) (23)) + ((VersionedChunkMeta) (7)) + ((SystemJobs) (8)) + ((Query) (9)) + ((TmpfsLayers) (11)) + ((MasterCache) (12)) + ((Lookup) (24)) + ((LookupRowsCache) (13)) + ((AllocFragmentation) (14)) + ((P2P) (15)) + ((Unknown) (17)) + ((Mixed) (18)) + ((TabletBackground) (19)) + ((JobInputBlockCache) (25)) + ((JobInputChunkMetaCache) (26)) + ((TableReplication) (27)) + ((ChaosReplicationIncoming) (28)) + ((ChaosReplicationOutgoing) (29)) + ((ReadTable) (30)) + ((Other) (31)) +); + +//////////////////////////////////////////////////////////////////////////////// + +} // namespace NYT diff --git a/yt/yt/ytlib/node_tracker_client/public.h b/yt/yt/ytlib/node_tracker_client/public.h new file mode 100644 index 0000000000..c642a3c530 --- /dev/null +++ b/yt/yt/ytlib/node_tracker_client/public.h @@ -0,0 +1,89 @@ +#pragma once + +#include <yt/yt/client/node_tracker_client/public.h> + +#include <yt/yt/ytlib/object_client/public.h> + +#include <yt/yt/ytlib/misc/public.h> + +namespace NYT::NNodeTrackerClient { + +//////////////////////////////////////////////////////////////////////////////// + +namespace NProto { + +class TReqRegisterNode; +class TRspRegisterNode; + +} // namespace NProto + +//////////////////////////////////////////////////////////////////////////////// + +class TNodeDirectoryBuilder; + +DECLARE_REFCOUNTED_STRUCT(INodeDirectorySynchronizer) + +DECLARE_REFCOUNTED_CLASS(TNodeDirectorySynchronizerConfig) + +DECLARE_REFCOUNTED_STRUCT(INodeChannelFactory) + +DECLARE_REFCOUNTED_STRUCT(INodeStatusDirectory) + +DEFINE_ENUM(EMemoryLimitType, + ((None) (0)) + ((Static) (1)) + ((Dynamic) (2)) +); + +DEFINE_ENUM(ENodeState, + // Used internally. + ((Unknown) (-1)) + // Not registered. + ((Offline) (0)) + // Registered but did not report some of the heartbeats. + ((Registered) (1)) + // Registered and reported all the expected types of heartbeats + // at least once. + ((Online) (2)) + // Unregistered and placed into disposal queue. + ((Unregistered) (3)) + // Indicates that state varies across cells. + ((Mixed) (4)) + // Unregistered and ongoing disposal. + ((BeingDisposed) (5)) +); + +DEFINE_ENUM(ECellAggregatedStateReliability, + // Used internally. + ((Unknown) (0)) + // Node knows about this cell from config. + ((StaticallyKnown) (1)) + // Indicates that node will receive information about this cell dynamically, + // no need to take into account information about node from cell, + // marked as ECellAggregatedStateReliability::DuringPropagation during computing aggregated state on primary master. + ((DuringPropagation) (2)) + // Indicates that node already received information about this cell dynamically. + ((DynamicallyDiscovered) (3)) +); + +DEFINE_ENUM(ENodeRole, + ((MasterCache) (0)) + ((TimestampProvider) (1)) +); + +DEFINE_ENUM(ENodeFlavor, + // COMPAT(gritukan) + ((Cluster) (0)) + // Node that is used to store chunks. + ((Data) (1)) + // Node that is used to execute jobs. + ((Exec) (2)) + // Node that is used to host dynamic tables tablets. + ((Tablet) (3)) + // Node that is used to host chaos cells. + ((Chaos) (4)) +); + +//////////////////////////////////////////////////////////////////////////////// + +} // namespace NYT::NNodeTrackerClient diff --git a/yt/yt/ytlib/object_client/public.h b/yt/yt/ytlib/object_client/public.h new file mode 100644 index 0000000000..d524b2c2ab --- /dev/null +++ b/yt/yt/ytlib/object_client/public.h @@ -0,0 +1,61 @@ +#pragma once + +#include <yt/yt/client/object_client/public.h> + +#include <library/cpp/yt/memory/ref_counted.h> + +namespace NYT::NObjectClient { + +//////////////////////////////////////////////////////////////////////////////// + +DECLARE_REFCOUNTED_CLASS(TReqExecuteBatchWithRetriesConfig) + +DECLARE_REFCOUNTED_CLASS(TObjectAttributeCacheConfig) +DECLARE_REFCOUNTED_CLASS(TObjectAttributeCache) + +DECLARE_REFCOUNTED_CLASS(TObjectServiceCacheConfig) +DECLARE_REFCOUNTED_CLASS(TObjectServiceCacheDynamicConfig) +DECLARE_REFCOUNTED_CLASS(TCachingObjectServiceConfig) +DECLARE_REFCOUNTED_CLASS(TCachingObjectServiceDynamicConfig) + +DECLARE_REFCOUNTED_CLASS(TObjectServiceCacheEntry) +DECLARE_REFCOUNTED_CLASS(TObjectServiceCache) + +DECLARE_REFCOUNTED_CLASS(TAbcConfig) + +DECLARE_REFCOUNTED_STRUCT(ICachingObjectService) + +//////////////////////////////////////////////////////////////////////////////// + +DEFINE_ENUM(EMasterFeature, + ((OverlayedJournals) (0)) + ((Portals) (1)) + ((PortalExitSynchronization) (2)) +); + +// Some objects must be created and removed atomically. +// +// Let's consider accounts. In the absence of an atomic commit, it's possible +// that some cell knows about an account, and some other cell doesn't. Then, the +// former cell sending a chunk requisition update to the latter will cause +// trouble. +// +// Removal also needs two-phase (and even more!) locking since otherwise a primary master +// is unable to command the destruction of an object to its secondaries without risking +// that some secondary still holds a reference to the object. +DEFINE_ENUM_WITH_UNDERLYING_TYPE(EObjectLifeStage, ui8, + // Creation workflow + ((CreationStarted) (0)) + ((CreationPreCommitted) (1)) + ((CreationCommitted) (2)) + + // Removal workflow + ((RemovalStarted) (3)) + ((RemovalPreCommitted) (4)) + ((RemovalAwaitingCellsSync)(5)) + ((RemovalCommitted) (6)) +); + +//////////////////////////////////////////////////////////////////////////////// + +} // namespace NYT::NObjectClient diff --git a/yt/yt/ytlib/table_client/public.h b/yt/yt/ytlib/table_client/public.h new file mode 100644 index 0000000000..aa337eec60 --- /dev/null +++ b/yt/yt/ytlib/table_client/public.h @@ -0,0 +1,170 @@ +#pragma once + +#include <yt/yt/client/table_client/public.h> + +namespace NYT::NTableClient { + +//////////////////////////////////////////////////////////////////////////////// + +namespace NProto { + +class TVirtualValueDirectory; + +} // namespace NProto + +constexpr int DefaultPartitionTag = -1; + +// TODO(ifsmirnov): calculate actual estimates. +constexpr i64 DefaultRemoteDynamicStoreReaderMemoryEstimate = 64_MB; + +DECLARE_REFCOUNTED_CLASS(TChunkColumnMapping) + +struct TColumnIdMapping +{ + int ChunkSchemaIndex; + int ReaderSchemaIndex; +}; + +DECLARE_REFCOUNTED_CLASS(TTableSchema) + +class TSchemaDictionary; + +template <typename TColumnName> +class TGenericColumnFilterDictionary; + +using TColumnFilterDictionary = TGenericColumnFilterDictionary<TString>; +using TColumnStableNameFilterDictionary = TGenericColumnFilterDictionary<TColumnStableName>; + +class THorizontalBlockReader; + +struct THunkChunkRef; +struct THunkChunksInfo; +struct THunkChunkMeta; + +struct TTableReadSpec; +struct TFetchSingleTableReadSpecOptions; + +struct TBoundaryKeysExtension; +struct TColumnMetaExtension; +struct TKeyColumnsExtension; +struct TSamplesExtension; + +DECLARE_REFCOUNTED_STRUCT(TOffloadingReaderOptions) +DECLARE_REFCOUNTED_STRUCT(IOffloadingReader) + +DECLARE_REFCOUNTED_CLASS(TSamplesFetcher) + +DECLARE_REFCOUNTED_STRUCT(IChunkSliceFetcher) + +DECLARE_REFCOUNTED_CLASS(TChunkSliceSizeFetcher) + +DECLARE_REFCOUNTED_CLASS(TKeySetWriter) + +DECLARE_REFCOUNTED_STRUCT(ISchemalessChunkReader) +DECLARE_REFCOUNTED_STRUCT(ISchemalessChunkWriter) + +DECLARE_REFCOUNTED_STRUCT(ISchemalessMultiChunkReader) +DECLARE_REFCOUNTED_STRUCT(ISchemalessMultiChunkWriter) + +DECLARE_REFCOUNTED_CLASS(TPartitionChunkReader) +DECLARE_REFCOUNTED_CLASS(TPartitionMultiChunkReader) + +DECLARE_REFCOUNTED_STRUCT(IVersionedChunkWriter) +DECLARE_REFCOUNTED_STRUCT(IVersionedMultiChunkWriter) + +DECLARE_REFCOUNTED_STRUCT(IHunkChunkPayloadWriter) + +DECLARE_REFCOUNTED_STRUCT(ITimingReader) + +DECLARE_REFCOUNTED_STRUCT(IPartitioner) + +DECLARE_REFCOUNTED_CLASS(TVersionedRowsetReader) + +DECLARE_REFCOUNTED_STRUCT(TXorFilterMeta) +DECLARE_REFCOUNTED_CLASS(TColumnarChunkMeta) +DECLARE_REFCOUNTED_CLASS(TCachedVersionedChunkMeta) + +DECLARE_REFCOUNTED_CLASS(TColumnarStatisticsFetcher) + +DECLARE_REFCOUNTED_STRUCT(TChunkReaderPerformanceCounters) + +DECLARE_REFCOUNTED_STRUCT(TChunkLookupHashTable) + +DECLARE_REFCOUNTED_STRUCT(TChunkState) + +DECLARE_REFCOUNTED_STRUCT(TTabletSnapshot) + +DECLARE_REFCOUNTED_STRUCT(TVirtualValueDirectory) + +DECLARE_REFCOUNTED_STRUCT(IVersionedRowDigestBuilder) + +struct TOwningBoundaryKeys; + +struct TBlobTableSchema; +class TBlobTableWriter; + +struct TChunkTimestamps; + +DECLARE_REFCOUNTED_CLASS(TSkynetColumnEvaluator) + +DECLARE_REFCOUNTED_CLASS(TCachedBlockMeta) +DECLARE_REFCOUNTED_CLASS(TBlockMetaCache) + +DECLARE_REFCOUNTED_CLASS(TTableColumnarStatisticsCache) + +class TSchemafulRowMerger; +class TUnversionedRowMerger; +class TSamplingRowMerger; + +struct IVersionedRowMerger; + +DECLARE_REFCOUNTED_CLASS(TTableWriterOptions) +DECLARE_REFCOUNTED_CLASS(TTableReaderOptions) + +DECLARE_REFCOUNTED_CLASS(TBlobTableWriterConfig) +DECLARE_REFCOUNTED_CLASS(TBufferedTableWriterConfig) +DECLARE_REFCOUNTED_CLASS(TPartitionConfig) +DECLARE_REFCOUNTED_CLASS(TTableColumnarStatisticsCacheConfig) +DECLARE_REFCOUNTED_CLASS(THunkChunkPayloadWriterConfig) + +DECLARE_REFCOUNTED_STRUCT(IHunkChunkReaderStatistics) +DECLARE_REFCOUNTED_STRUCT(IHunkChunkWriterStatistics) +class THunkChunkReaderCounters; +class THunkChunkWriterCounters; + +class TSliceBoundaryKey; + +DEFINE_ENUM(ETableCollocationType, + ((Replication) (0)) +); + +DECLARE_REFCOUNTED_STRUCT(IChunkIndexBuilder) + +DECLARE_REFCOUNTED_STRUCT(IKeyFilter) +DECLARE_REFCOUNTED_STRUCT(IKeyFilterBuilder) + +DECLARE_REFCOUNTED_STRUCT(TKeyFilterStatistics) + +constexpr int VersionedBlockValueSize = 16; + +constexpr int IndexedRowTypicalGroupCount = 1; + +class TIndexedVersionedBlockFormatDetail; + +DECLARE_REFCOUNTED_STRUCT(IChunkIndexReadController) + +DECLARE_REFCOUNTED_STRUCT(TTabletPerformanceCounters) + +DECLARE_REFCOUNTED_STRUCT(IGranuleFilter) + +struct TVersionedRowDigest; + +DECLARE_REFCOUNTED_STRUCT(IDictionaryCompressionSession) +DECLARE_REFCOUNTED_STRUCT(IDictionaryDecompressionSession) +DECLARE_REFCOUNTED_STRUCT(IDictionaryCompressionFactory) + +struct TTimestampReadOptions; + +//////////////////////////////////////////////////////////////////////////////// + +} // namespace NYT::NTableClient diff --git a/yt/yt/ytlib/table_client/table_upload_options.h b/yt/yt/ytlib/table_client/table_upload_options.h new file mode 100644 index 0000000000..20cb95440d --- /dev/null +++ b/yt/yt/ytlib/table_client/table_upload_options.h @@ -0,0 +1,87 @@ +#pragma once + +#include "public.h" + +#include <yt/yt/ytlib/chunk_client/public.h> + +#include <yt/yt/client/table_client/schema.h> +#include <yt/yt/client/table_client/versioned_io_options.h> + +#include <yt/yt/client/security_client/public.h> + +#include <yt/yt/library/erasure/public.h> + +#include <yt/yt/core/compression/public.h> + +#include <yt/yt/core/misc/phoenix.h> + +namespace NYT::NTableClient { + +//////////////////////////////////////////////////////////////////////////////// + +class TEpochSchema +{ +public: + TEpochSchema() = default; + TEpochSchema(const TEpochSchema& other); + TEpochSchema& operator=(const TEpochSchema& other); + + TEpochSchema(const TEpochSchema&& other) = delete; + TEpochSchema& operator=(TEpochSchema&& other) = delete; + + TEpochSchema& operator=(TTableSchemaPtr schema); + + const TTableSchema* operator->() const; + const TTableSchemaPtr& operator*() const; + + const TTableSchemaPtr& Get() const; + ui64 GetRevision() const; + + ui64 Set(const TTableSchemaPtr& schema); + + void Persist(const NPhoenix::TPersistenceContext& context); + + ui64 Reset(); + +private: + TTableSchemaPtr TableSchema_ = New<TTableSchema>(); + ui64 Revision_ = 0; +}; + +struct TTableUploadOptions +{ + NChunkClient::EUpdateMode UpdateMode; + NCypressClient::ELockMode LockMode; + TEpochSchema TableSchema; + TMasterTableSchemaId SchemaId; + ETableSchemaModification SchemaModification; + TVersionedWriteOptions VersionedWriteOptions; + ETableSchemaMode SchemaMode; + EOptimizeFor OptimizeFor; + std::optional<NChunkClient::EChunkFormat> ChunkFormat; + NCompression::ECodec CompressionCodec; + NErasure::ECodec ErasureCodec; + bool EnableStripedErasure; + std::optional<std::vector<NSecurityClient::TSecurityTag>> SecurityTags; + bool PartiallySorted; + + TTableSchemaPtr GetUploadSchema() const; + + void Persist(const NPhoenix::TPersistenceContext& context); +}; + +const std::vector<TString>& GetTableUploadOptionsAttributeKeys(); + +TTableUploadOptions GetTableUploadOptions( + const NYPath::TRichYPath& path, + const NYTree::IAttributeDictionary& cypressTableAttributes, + const TTableSchemaPtr& schema, + i64 rowCount); + +TTableUploadOptions GetFileUploadOptions( + const NYPath::TRichYPath& path, + const NYTree::IAttributeDictionary& cypressTableAttributes); + +//////////////////////////////////////////////////////////////////////////////// + +} // namespace NYT::NTableClient |