mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-28 22:45:26 +08:00
enhance: support specified version manifest write (#46331)
Related to #44956 **Support specified version manifest write** - Add `baseVersion` parameter to `NewPackedRecordManifestWriter` and `NewFFIPackedWriter` to support writing manifest based on a specific version instead of always overwriting the latest - Add `manifestPath` tracking in `BulkPackWriterV2` to maintain manifest state across writes - Add `GetManifestInfo` method to parse existing manifest path and extract base path and version - Add `UpdateManifestPath` metacache action to track manifest path in segment info - Update `transaction_begin` FFI call to use the specified base version --------- Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
This commit is contained in:
parent
971085b033
commit
18fbaaca0a
@ -290,7 +290,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) initStorageV2Segments(rows int,
|
|||||||
bw := syncmgr.NewBulkPackWriterV2(mc, sch, cm, s.mockAlloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
|
bw := syncmgr.NewBulkPackWriterV2(mc, sch, cm, s.mockAlloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
|
||||||
StorageType: "local",
|
StorageType: "local",
|
||||||
RootPath: rootPath,
|
RootPath: rootPath,
|
||||||
}, columnGroups)
|
}, columnGroups, "")
|
||||||
return bw.Write(context.Background(), pack)
|
return bw.Write(context.Background(), pack)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -333,7 +333,7 @@ func (s *MixCompactionTaskStorageV2Suite) initStorageV2Segments(rows int, seed i
|
|||||||
bw := syncmgr.NewBulkPackWriterV2(mc, s.meta.Schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
|
bw := syncmgr.NewBulkPackWriterV2(mc, s.meta.Schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
|
||||||
StorageType: "local",
|
StorageType: "local",
|
||||||
RootPath: rootPath,
|
RootPath: rootPath,
|
||||||
}, columnGroups)
|
}, columnGroups, "")
|
||||||
return bw.Write(context.Background(), pack)
|
return bw.Write(context.Background(), pack)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -111,7 +111,7 @@ func (s *NamespaceCompactorTestSuite) setupSortedSegments() {
|
|||||||
bw := syncmgr.NewBulkPackWriterV2(mc, s.schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
|
bw := syncmgr.NewBulkPackWriterV2(mc, s.schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
|
||||||
StorageType: "local",
|
StorageType: "local",
|
||||||
RootPath: rootPath,
|
RootPath: rootPath,
|
||||||
}, columnGroups)
|
}, columnGroups, "")
|
||||||
inserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
inserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||||
s.Require().NoError(err)
|
s.Require().NoError(err)
|
||||||
s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{
|
s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{
|
||||||
|
|||||||
@ -243,6 +243,12 @@ func SetStartPosRecorded(flag bool) SegmentAction {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func UpdateManifestPath(manifestPath string) SegmentAction {
|
||||||
|
return func(info *SegmentInfo) {
|
||||||
|
info.manifestPath = manifestPath
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// MergeSegmentAction is the util function to merge multiple SegmentActions into one.
|
// MergeSegmentAction is the util function to merge multiple SegmentActions into one.
|
||||||
func MergeSegmentAction(actions ...SegmentAction) SegmentAction {
|
func MergeSegmentAction(actions ...SegmentAction) SegmentAction {
|
||||||
return func(info *SegmentInfo) {
|
return func(info *SegmentInfo) {
|
||||||
|
|||||||
@ -31,6 +31,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/flushcommon/metacache"
|
"github.com/milvus-io/milvus/internal/flushcommon/metacache"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
"github.com/milvus-io/milvus/internal/storagecommon"
|
"github.com/milvus-io/milvus/internal/storagecommon"
|
||||||
|
"github.com/milvus-io/milvus/internal/storagev2/packed"
|
||||||
"github.com/milvus-io/milvus/internal/util/hookutil"
|
"github.com/milvus-io/milvus/internal/util/hookutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
@ -51,11 +52,12 @@ type BulkPackWriterV2 struct {
|
|||||||
|
|
||||||
storageConfig *indexpb.StorageConfig
|
storageConfig *indexpb.StorageConfig
|
||||||
columnGroups []storagecommon.ColumnGroup
|
columnGroups []storagecommon.ColumnGroup
|
||||||
|
manifestPath string
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewBulkPackWriterV2(metaCache metacache.MetaCache, schema *schemapb.CollectionSchema, chunkManager storage.ChunkManager,
|
func NewBulkPackWriterV2(metaCache metacache.MetaCache, schema *schemapb.CollectionSchema, chunkManager storage.ChunkManager,
|
||||||
allocator allocator.Interface, bufferSize, multiPartUploadSize int64,
|
allocator allocator.Interface, bufferSize, multiPartUploadSize int64,
|
||||||
storageConfig *indexpb.StorageConfig, columnGroups []storagecommon.ColumnGroup, writeRetryOpts ...retry.Option,
|
storageConfig *indexpb.StorageConfig, columnGroups []storagecommon.ColumnGroup, curManifestPath string, writeRetryOpts ...retry.Option,
|
||||||
) *BulkPackWriterV2 {
|
) *BulkPackWriterV2 {
|
||||||
return &BulkPackWriterV2{
|
return &BulkPackWriterV2{
|
||||||
BulkPackWriter: &BulkPackWriter{
|
BulkPackWriter: &BulkPackWriter{
|
||||||
@ -70,6 +72,7 @@ func NewBulkPackWriterV2(metaCache metacache.MetaCache, schema *schemapb.Collect
|
|||||||
multiPartUploadSize: multiPartUploadSize,
|
multiPartUploadSize: multiPartUploadSize,
|
||||||
storageConfig: storageConfig,
|
storageConfig: storageConfig,
|
||||||
columnGroups: columnGroups,
|
columnGroups: columnGroups,
|
||||||
|
manifestPath: curManifestPath,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -179,6 +182,27 @@ func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (m
|
|||||||
return logs, manifestPath, nil
|
return logs, manifestPath, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (bw *BulkPackWriterV2) GetManifestInfo(pack *SyncPack) (basePath string, version int64, err error) {
|
||||||
|
// empty info, shall be first write,
|
||||||
|
// initialize manifestPath with -1 version
|
||||||
|
if bw.manifestPath == "" {
|
||||||
|
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID)
|
||||||
|
logicalPath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k)
|
||||||
|
bucketName := bw.getBucketName()
|
||||||
|
// if storage config is not passed, use common config
|
||||||
|
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
|
||||||
|
if bw.storageConfig != nil {
|
||||||
|
storageType = bw.storageConfig.GetStorageType()
|
||||||
|
}
|
||||||
|
if storageType != "local" {
|
||||||
|
basePath = path.Join(bucketName, logicalPath)
|
||||||
|
}
|
||||||
|
return basePath, -1, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
return packed.UnmarshalManfestPath(bw.manifestPath)
|
||||||
|
}
|
||||||
|
|
||||||
func (bw *BulkPackWriterV2) writeInsertsIntoStorage(_ context.Context,
|
func (bw *BulkPackWriterV2) writeInsertsIntoStorage(_ context.Context,
|
||||||
pluginContextPtr *indexcgopb.StoragePluginContext,
|
pluginContextPtr *indexcgopb.StoragePluginContext,
|
||||||
pack *SyncPack,
|
pack *SyncPack,
|
||||||
@ -203,10 +227,12 @@ func (bw *BulkPackWriterV2) writeInsertsIntoStorage(_ context.Context,
|
|||||||
}
|
}
|
||||||
|
|
||||||
var manifestPath string
|
var manifestPath string
|
||||||
if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() {
|
if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() || bw.manifestPath != "" {
|
||||||
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID)
|
basePath, version, err := bw.GetManifestInfo(pack)
|
||||||
basePath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k)
|
if err != nil {
|
||||||
w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
|
return nil, "", err
|
||||||
|
}
|
||||||
|
w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, version, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, "", err
|
return nil, "", err
|
||||||
}
|
}
|
||||||
|
|||||||
@ -143,7 +143,7 @@ func (s *PackWriterV2Suite) TestPackWriterV2_Write() {
|
|||||||
|
|
||||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema)).WithDeleteData(deletes)
|
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema)).WithDeleteData(deletes)
|
||||||
|
|
||||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
|
||||||
|
|
||||||
gotInserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
gotInserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||||
s.NoError(err)
|
s.NoError(err)
|
||||||
@ -162,7 +162,7 @@ func (s *PackWriterV2Suite) TestWriteEmptyInsertData() {
|
|||||||
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
|
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
|
||||||
|
|
||||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName)
|
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName)
|
||||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
|
||||||
|
|
||||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||||
s.NoError(err)
|
s.NoError(err)
|
||||||
@ -191,7 +191,7 @@ func (s *PackWriterV2Suite) TestNoPkField() {
|
|||||||
buf.Append(data)
|
buf.Append(data)
|
||||||
|
|
||||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
|
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
|
||||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
|
||||||
|
|
||||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||||
s.Error(err)
|
s.Error(err)
|
||||||
@ -208,7 +208,7 @@ func (s *PackWriterV2Suite) TestAllocIDExhausedError() {
|
|||||||
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
|
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
|
||||||
|
|
||||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema))
|
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema))
|
||||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
|
||||||
|
|
||||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||||
s.Error(err)
|
s.Error(err)
|
||||||
@ -229,7 +229,7 @@ func (s *PackWriterV2Suite) TestWriteInsertDataError() {
|
|||||||
buf.Append(data)
|
buf.Append(data)
|
||||||
|
|
||||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
|
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
|
||||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
|
||||||
|
|
||||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||||
s.Error(err)
|
s.Error(err)
|
||||||
|
|||||||
@ -136,10 +136,9 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
|
|||||||
|
|
||||||
switch segmentInfo.GetStorageVersion() {
|
switch segmentInfo.GetStorageVersion() {
|
||||||
case storage.StorageV2:
|
case storage.StorageV2:
|
||||||
// TODO change to return manifest after integrated
|
|
||||||
// New sync task means needs to flush data immediately, so do not need to buffer data in writer again.
|
// New sync task means needs to flush data immediately, so do not need to buffer data in writer again.
|
||||||
writer := NewBulkPackWriterV2(t.metacache, t.schema, t.chunkManager, t.allocator, 0,
|
writer := NewBulkPackWriterV2(t.metacache, t.schema, t.chunkManager, t.allocator, 0,
|
||||||
packed.DefaultMultiPartUploadSize, t.storageConfig, columnGroups, t.writeRetryOpts...)
|
packed.DefaultMultiPartUploadSize, t.storageConfig, columnGroups, segmentInfo.ManifestPath(), t.writeRetryOpts...)
|
||||||
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.manifestPath, t.flushedSize, err = writer.Write(ctx, t.pack)
|
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.manifestPath, t.flushedSize, err = writer.Write(ctx, t.pack)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("failed to write sync data with storage v2 format", zap.Error(err))
|
log.Warn("failed to write sync data with storage v2 format", zap.Error(err))
|
||||||
@ -181,7 +180,7 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
|
|||||||
|
|
||||||
t.pack.ReleaseData()
|
t.pack.ReleaseData()
|
||||||
|
|
||||||
actions := []metacache.SegmentAction{metacache.FinishSyncing(t.batchRows)}
|
actions := []metacache.SegmentAction{metacache.FinishSyncing(t.batchRows), metacache.UpdateManifestPath(t.manifestPath)}
|
||||||
if columnGroups != nil {
|
if columnGroups != nil {
|
||||||
actions = append(actions, metacache.UpdateCurrentSplit(columnGroups))
|
actions = append(actions, metacache.UpdateCurrentSplit(columnGroups))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -361,7 +361,10 @@ func (pw *PackedManifestRecordWriter) initWriters(r Record) error {
|
|||||||
var err error
|
var err error
|
||||||
k := metautil.JoinIDPath(pw.collectionID, pw.partitionID, pw.segmentID)
|
k := metautil.JoinIDPath(pw.collectionID, pw.partitionID, pw.segmentID)
|
||||||
basePath := path.Join(pw.storageConfig.GetRootPath(), common.SegmentInsertLogPath, k)
|
basePath := path.Join(pw.storageConfig.GetRootPath(), common.SegmentInsertLogPath, k)
|
||||||
pw.writer, err = NewPackedRecordManifestWriter(pw.storageConfig.GetBucketName(), basePath, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
|
if pw.storageConfig.StorageType != "local" {
|
||||||
|
basePath = path.Join(pw.storageConfig.GetBucketName(), basePath)
|
||||||
|
}
|
||||||
|
pw.writer, err = NewPackedRecordManifestWriter(pw.storageConfig.GetBucketName(), basePath, -1, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -296,6 +296,7 @@ func (pw *packedRecordManifestWriter) Close() error {
|
|||||||
func NewPackedRecordManifestWriter(
|
func NewPackedRecordManifestWriter(
|
||||||
bucketName string,
|
bucketName string,
|
||||||
basePath string,
|
basePath string,
|
||||||
|
baseVersion int64,
|
||||||
schema *schemapb.CollectionSchema,
|
schema *schemapb.CollectionSchema,
|
||||||
bufferSize int64,
|
bufferSize int64,
|
||||||
multiPartUploadSize int64,
|
multiPartUploadSize int64,
|
||||||
@ -314,16 +315,8 @@ func NewPackedRecordManifestWriter(
|
|||||||
return nil, merr.WrapErrServiceInternal(
|
return nil, merr.WrapErrServiceInternal(
|
||||||
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
|
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
|
||||||
}
|
}
|
||||||
// if storage config is not passed, use common config
|
|
||||||
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
|
writer, err := packed.NewFFIPackedWriter(basePath, baseVersion, arrowSchema, columnGroups, storageConfig, storagePluginContext)
|
||||||
if storageConfig != nil {
|
|
||||||
storageType = storageConfig.GetStorageType()
|
|
||||||
}
|
|
||||||
ffiBasePath := basePath
|
|
||||||
if storageType != "local" {
|
|
||||||
ffiBasePath = path.Join(bucketName, basePath)
|
|
||||||
}
|
|
||||||
writer, err := packed.NewFFIPackedWriter(ffiBasePath, arrowSchema, columnGroups, storageConfig, storagePluginContext)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, merr.WrapErrServiceInternal(
|
return nil, merr.WrapErrServiceInternal(
|
||||||
fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||||
|
|||||||
@ -74,7 +74,7 @@ func createStorageConfig() *indexpb.StorageConfig {
|
|||||||
return storageConfig
|
return storageConfig
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewFFIPackedWriter(basePath string, schema *arrow.Schema, columnGroups []storagecommon.ColumnGroup, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*FFIPackedWriter, error) {
|
func NewFFIPackedWriter(basePath string, baseVersion int64, schema *arrow.Schema, columnGroups []storagecommon.ColumnGroup, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*FFIPackedWriter, error) {
|
||||||
cBasePath := C.CString(basePath)
|
cBasePath := C.CString(basePath)
|
||||||
defer C.free(unsafe.Pointer(cBasePath))
|
defer C.free(unsafe.Pointer(cBasePath))
|
||||||
|
|
||||||
@ -143,6 +143,7 @@ func NewFFIPackedWriter(basePath string, schema *arrow.Schema, columnGroups []st
|
|||||||
|
|
||||||
return &FFIPackedWriter{
|
return &FFIPackedWriter{
|
||||||
basePath: basePath,
|
basePath: basePath,
|
||||||
|
baseVersion: baseVersion,
|
||||||
cWriterHandle: writerHandle,
|
cWriterHandle: writerHandle,
|
||||||
cProperties: cProperties,
|
cProperties: cProperties,
|
||||||
}, nil
|
}, nil
|
||||||
@ -178,7 +179,7 @@ func (pw *FFIPackedWriter) Close() (string, error) {
|
|||||||
|
|
||||||
// TODO pass version
|
// TODO pass version
|
||||||
// use -1 as latest
|
// use -1 as latest
|
||||||
result = C.transaction_begin(cBasePath, pw.cProperties, &transationHandle, C.int64_t(-1))
|
result = C.transaction_begin(cBasePath, pw.cProperties, &transationHandle, C.int64_t(pw.baseVersion))
|
||||||
if err := HandleFFIResult(result); err != nil {
|
if err := HandleFFIResult(result); err != nil {
|
||||||
return "", err
|
return "", err
|
||||||
}
|
}
|
||||||
|
|||||||
@ -36,6 +36,7 @@ type PackedWriter struct {
|
|||||||
|
|
||||||
type FFIPackedWriter struct {
|
type FFIPackedWriter struct {
|
||||||
basePath string
|
basePath string
|
||||||
|
baseVersion int64
|
||||||
cWriterHandle C.WriterHandle
|
cWriterHandle C.WriterHandle
|
||||||
cProperties *C.Properties
|
cProperties *C.Properties
|
||||||
}
|
}
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user