enhance: move object storage client creation to pkg package (#40440)

issue: #40439

Signed-off-by: jaime <yun.zhang@zilliz.com>
This commit is contained in:
jaime 2025-03-12 20:38:07 +08:00 committed by GitHub
parent 27c7cbbc72
commit c8a96377bb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
38 changed files with 774 additions and 605 deletions

6
go.mod
View File

@ -6,9 +6,9 @@ toolchain go1.22.7
require (
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 // indirect
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0
github.com/aliyun/credentials-go v1.2.7
github.com/aliyun/credentials-go v1.2.7 // indirect
github.com/apache/arrow/go/v17 v17.0.0
github.com/bits-and-blooms/bloom/v3 v3.0.1
github.com/blang/semver/v4 v4.0.0
@ -36,7 +36,7 @@ require (
github.com/spf13/cast v1.3.1
github.com/spf13/viper v1.8.1
github.com/stretchr/testify v1.9.0
github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.865
github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.865 // indirect
github.com/tikv/client-go/v2 v2.0.4
go.etcd.io/etcd/api/v3 v3.5.5
go.etcd.io/etcd/client/v3 v3.5.5

View File

@ -37,6 +37,7 @@ import (
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
@ -60,7 +61,7 @@ type ClusteringCompactionTaskSuite struct {
func (s *ClusteringCompactionTaskSuite) SetupTest() {
ctx := context.Background()
cm := storage.NewLocalChunkManager(storage.RootPath(""))
cm := storage.NewLocalChunkManager(objectstorage.RootPath(""))
catalog := datacoord.NewCatalog(NewMetaMemoryKV(), "", "")
broker := broker.NewMockBroker(s.T())
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil)

View File

@ -42,6 +42,7 @@ import (
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/mq/msgdispatcher"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
@ -127,7 +128,7 @@ func TestDataNode(t *testing.T) {
defer node.Stop()
node.chunkManager = storage.NewLocalChunkManager(storage.RootPath("/tmp/milvus_test/datanode"))
node.chunkManager = storage.NewLocalChunkManager(objectstorage.RootPath("/tmp/milvus_test/datanode"))
paramtable.SetNodeID(1)
defer cancel()

View File

@ -43,6 +43,7 @@ import (
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
@ -109,7 +110,7 @@ func (s *DataNodeServicesSuite) SetupTest() {
err = s.node.Start()
s.Require().NoError(err)
s.node.chunkManager = storage.NewLocalChunkManager(storage.RootPath("/tmp/milvus_test/datanode"))
s.node.chunkManager = storage.NewLocalChunkManager(objectstorage.RootPath("/tmp/milvus_test/datanode"))
paramtable.SetNodeID(1)
}

View File

@ -9,6 +9,7 @@ import (
"golang.org/x/net/context"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
@ -27,7 +28,7 @@ type BinlogIOSuite struct {
func (s *BinlogIOSuite) SetupTest() {
paramtable.Init()
s.cm = storage.NewLocalChunkManager(storage.RootPath(binlogIOTestDir))
s.cm = storage.NewLocalChunkManager(objectstorage.RootPath(binlogIOTestDir))
s.b = NewBinlogIO(s.cm)
}

View File

@ -44,6 +44,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/mq/msgdispatcher"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
@ -167,7 +168,7 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
"add un-flushed and flushed segments",
},
}
cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
cm := storage.NewLocalChunkManager(objectstorage.RootPath(dataSyncServiceTestDir))
defer cm.RemoveWithPrefix(ctx, cm.RootPath())
wbManager := writebuffer.NewMockBufferManager(t)
@ -238,7 +239,7 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
func TestGetChannelWithTickler(t *testing.T) {
channelName := "by-dev-rootcoord-dml-0"
info := GetWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch)
chunkManager := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
chunkManager := storage.NewLocalChunkManager(objectstorage.RootPath(dataSyncServiceTestDir))
defer chunkManager.RemoveWithPrefix(context.Background(), chunkManager.RootPath())
meta := NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64)

View File

@ -38,6 +38,7 @@ import (
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/internal/util/initcore"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
@ -89,7 +90,7 @@ func (s *PackWriterV2Suite) SetupTest() {
},
},
}
s.cm = storage.NewLocalChunkManager(storage.RootPath(s.rootPath))
s.cm = storage.NewLocalChunkManager(objectstorage.RootPath(s.rootPath))
}
func (s *PackWriterV2Suite) TestPackWriterV2_Write() {

View File

@ -5,6 +5,7 @@ import (
"fmt"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
@ -25,21 +26,21 @@ func NewChunkMgrFactory() *chunkMgrFactory {
func (m *chunkMgrFactory) NewChunkManager(ctx context.Context, config *indexpb.StorageConfig) (storage.ChunkManager, error) {
chunkManagerFactory := storage.NewChunkManagerFactory(config.GetStorageType(),
storage.RootPath(config.GetRootPath()),
storage.Address(config.GetAddress()),
storage.AccessKeyID(config.GetAccessKeyID()),
storage.SecretAccessKeyID(config.GetSecretAccessKey()),
storage.UseSSL(config.GetUseSSL()),
storage.SslCACert(config.GetSslCACert()),
storage.BucketName(config.GetBucketName()),
storage.UseIAM(config.GetUseIAM()),
storage.CloudProvider(config.GetCloudProvider()),
storage.IAMEndpoint(config.GetIAMEndpoint()),
storage.UseVirtualHost(config.GetUseVirtualHost()),
storage.RequestTimeout(config.GetRequestTimeoutMs()),
storage.Region(config.GetRegion()),
storage.CreateBucket(true),
storage.GcpCredentialJSON(config.GetGcpCredentialJSON()),
objectstorage.RootPath(config.GetRootPath()),
objectstorage.Address(config.GetAddress()),
objectstorage.AccessKeyID(config.GetAccessKeyID()),
objectstorage.SecretAccessKeyID(config.GetSecretAccessKey()),
objectstorage.UseSSL(config.GetUseSSL()),
objectstorage.SslCACert(config.GetSslCACert()),
objectstorage.BucketName(config.GetBucketName()),
objectstorage.UseIAM(config.GetUseIAM()),
objectstorage.CloudProvider(config.GetCloudProvider()),
objectstorage.IAMEndpoint(config.GetIAMEndpoint()),
objectstorage.UseVirtualHost(config.GetUseVirtualHost()),
objectstorage.RequestTimeout(config.GetRequestTimeoutMs()),
objectstorage.Region(config.GetRegion()),
objectstorage.CreateBucket(true),
objectstorage.GcpCredentialJSON(config.GetGcpCredentialJSON()),
)
return chunkManagerFactory.NewPersistentStorageChunkManager(ctx)
}

View File

@ -34,12 +34,13 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
storage "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
"github.com/milvus-io/milvus/internal/util/segcore"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/etcdpb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
@ -435,17 +436,17 @@ func GenTestIndexMeta(collectionID int64, schema *schemapb.CollectionSchema) *se
func NewTestChunkManagerFactory(params *paramtable.ComponentParam, rootPath string) *storage.ChunkManagerFactory {
return storage.NewChunkManagerFactory("minio",
storage.RootPath(rootPath),
storage.Address(params.MinioCfg.Address.GetValue()),
storage.AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
storage.SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
storage.UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
storage.SslCACert(params.MinioCfg.SslCACert.GetValue()),
storage.BucketName(params.MinioCfg.BucketName.GetValue()),
storage.UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
storage.CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
storage.IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
storage.CreateBucket(true))
objectstorage.RootPath(rootPath),
objectstorage.Address(params.MinioCfg.Address.GetValue()),
objectstorage.AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
objectstorage.SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
objectstorage.UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
objectstorage.SslCACert(params.MinioCfg.SslCACert.GetValue()),
objectstorage.BucketName(params.MinioCfg.BucketName.GetValue()),
objectstorage.UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
objectstorage.CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
objectstorage.IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
objectstorage.CreateBucket(true))
}
func SaveBinLog(ctx context.Context,

View File

@ -28,6 +28,7 @@ import (
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
@ -67,7 +68,7 @@ func (suite *HandlersSuite) SetupTest() {
// mock factory
suite.factory = dependency.NewMockFactory(suite.T())
suite.chunkManagerFactory = storage.NewChunkManagerFactory("local", storage.RootPath("/tmp/milvus_test"))
suite.chunkManagerFactory = storage.NewChunkManagerFactory("local", objectstorage.RootPath("/tmp/milvus_test"))
// new node
suite.node = NewQueryNode(context.Background(), suite.factory)

View File

@ -37,6 +37,7 @@ import (
"github.com/milvus-io/milvus/internal/querynodev2/segments"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
@ -70,7 +71,7 @@ func (suite *QueryNodeSuite) SetupTest() {
// mock factory
suite.factory = dependency.NewMockFactory(suite.T())
suite.chunkManagerFactory = storage.NewChunkManagerFactory("local", storage.RootPath("/tmp/milvus_test"))
suite.chunkManagerFactory = storage.NewChunkManagerFactory("local", objectstorage.RootPath("/tmp/milvus_test"))
// new node
suite.node = NewQueryNode(context.Background(), suite.factory)
// init etcd

View File

@ -19,71 +19,24 @@ package storage
import (
"context"
"io"
"os"
"time"
"github.com/Azure/azure-sdk-for-go/sdk/azcore"
"github.com/Azure/azure-sdk-for-go/sdk/azidentity"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/blob"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/bloberror"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/blockblob"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/container"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/service"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/retry"
)
type AzureObjectStorage struct {
*service.Client
}
func newAzureObjectStorageWithConfig(ctx context.Context, c *config) (*AzureObjectStorage, error) {
var client *service.Client
var err error
if c.useIAM {
cred, credErr := azidentity.NewWorkloadIdentityCredential(&azidentity.WorkloadIdentityCredentialOptions{
ClientID: os.Getenv("AZURE_CLIENT_ID"),
TenantID: os.Getenv("AZURE_TENANT_ID"),
TokenFilePath: os.Getenv("AZURE_FEDERATED_TOKEN_FILE"),
})
if credErr != nil {
return nil, credErr
}
client, err = service.NewClient("https://"+c.accessKeyID+".blob."+c.address+"/", cred, &service.ClientOptions{})
} else {
connectionString := os.Getenv("AZURE_STORAGE_CONNECTION_STRING")
if connectionString == "" {
connectionString = "DefaultEndpointsProtocol=https;AccountName=" + c.accessKeyID +
";AccountKey=" + c.secretAccessKeyID + ";EndpointSuffix=" + c.address
}
client, err = service.NewClientFromConnectionString(connectionString, &service.ClientOptions{})
}
if err != nil {
return nil, err
}
if c.bucketName == "" {
return nil, merr.WrapErrParameterInvalidMsg("invalid empty bucket name")
}
// check valid in first query
checkBucketFn := func() error {
_, err := client.NewContainerClient(c.bucketName).GetProperties(ctx, &container.GetPropertiesOptions{})
if err != nil {
switch err := err.(type) {
case *azcore.ResponseError:
if c.createBucket && err.ErrorCode == string(bloberror.ContainerNotFound) {
_, createErr := client.NewContainerClient(c.bucketName).Create(ctx, &azblob.CreateContainerOptions{})
if createErr != nil {
return createErr
}
return nil
}
}
}
return err
}
err = retry.Do(ctx, checkBucketFn, retry.Attempts(CheckBucketRetryAttempts))
func newAzureObjectStorageWithConfig(ctx context.Context, c *objectstorage.Config) (*AzureObjectStorage, error) {
client, err := objectstorage.NewAzureObjectStorageClient(ctx, c)
if err != nil {
return nil, err
}

View File

@ -27,24 +27,26 @@ import (
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
)
func TestAzureObjectStorage(t *testing.T) {
ctx := context.Background()
bucketName := Params.MinioCfg.BucketName.GetValue()
config := config{
bucketName: bucketName,
createBucket: true,
useIAM: false,
cloudProvider: "azure",
config := objectstorage.Config{
BucketName: bucketName,
CreateBucket: true,
UseIAM: false,
CloudProvider: "azure",
}
t.Run("test initialize", func(t *testing.T) {
var err error
config.bucketName = ""
config.BucketName = ""
_, err = newAzureObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.bucketName = bucketName
config.BucketName = bucketName
_, err = newAzureObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
})
@ -52,7 +54,7 @@ func TestAzureObjectStorage(t *testing.T) {
t.Run("test load", func(t *testing.T) {
testCM, err := newAzureObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
defer testCM.DeleteContainer(ctx, config.bucketName, &azblob.DeleteContainerOptions{})
defer testCM.DeleteContainer(ctx, config.BucketName, &azblob.DeleteContainerOptions{})
prepareTests := []struct {
key string
@ -66,7 +68,7 @@ func TestAzureObjectStorage(t *testing.T) {
}
for _, test := range prepareTests {
err := testCM.PutObject(ctx, config.bucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
err := testCM.PutObject(ctx, config.BucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
require.NoError(t, err)
}
@ -89,19 +91,19 @@ func TestAzureObjectStorage(t *testing.T) {
for _, test := range loadTests {
t.Run(test.description, func(t *testing.T) {
if test.isvalid {
got, err := testCM.GetObject(ctx, config.bucketName, test.loadKey, 0, 1024)
got, err := testCM.GetObject(ctx, config.BucketName, test.loadKey, 0, 1024)
assert.NoError(t, err)
contentData, err := io.ReadAll(got)
assert.NoError(t, err)
assert.Equal(t, len(contentData), len(test.expectedValue))
assert.Equal(t, test.expectedValue, contentData)
statSize, err := testCM.StatObject(ctx, config.bucketName, test.loadKey)
statSize, err := testCM.StatObject(ctx, config.BucketName, test.loadKey)
assert.NoError(t, err)
assert.Equal(t, statSize, int64(len(contentData)))
_, err = testCM.GetObject(ctx, config.bucketName, test.loadKey, 1, 1023)
_, err = testCM.GetObject(ctx, config.BucketName, test.loadKey, 1, 1023)
assert.NoError(t, err)
} else {
got, err := testCM.GetObject(ctx, config.bucketName, test.loadKey, 0, 1024)
got, err := testCM.GetObject(ctx, config.BucketName, test.loadKey, 0, 1024)
assert.NoError(t, err)
assert.NotEmpty(t, got)
_, err = io.ReadAll(got)
@ -124,11 +126,11 @@ func TestAzureObjectStorage(t *testing.T) {
for _, test := range loadWithPrefixTests {
t.Run(test.description, func(t *testing.T) {
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.bucketName, test.prefix, false)
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.BucketName, test.prefix, false)
assert.NoError(t, err)
assert.Equal(t, len(test.expectedValue), len(gotk))
for _, key := range gotk {
err := testCM.RemoveObject(ctx, config.bucketName, key)
err := testCM.RemoveObject(ctx, config.BucketName, key)
assert.NoError(t, err)
}
})
@ -138,7 +140,7 @@ func TestAzureObjectStorage(t *testing.T) {
t.Run("test list", func(t *testing.T) {
testCM, err := newAzureObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
defer testCM.DeleteContainer(ctx, config.bucketName, &azblob.DeleteContainerOptions{})
defer testCM.DeleteContainer(ctx, config.BucketName, &azblob.DeleteContainerOptions{})
prepareTests := []struct {
valid bool
@ -156,10 +158,10 @@ func TestAzureObjectStorage(t *testing.T) {
}
for _, test := range prepareTests {
err := testCM.PutObject(ctx, config.bucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
err := testCM.PutObject(ctx, config.BucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
require.Nil(t, err)
if !test.valid {
err := testCM.RemoveObject(ctx, config.bucketName, test.key)
err := testCM.RemoveObject(ctx, config.BucketName, test.key)
require.Nil(t, err)
}
}
@ -177,7 +179,7 @@ func TestAzureObjectStorage(t *testing.T) {
for _, test := range insertWithPrefixTests {
t.Run(fmt.Sprintf("prefix: %s, recursive: %t", test.prefix, test.recursive), func(t *testing.T) {
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.bucketName, test.prefix, test.recursive)
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.BucketName, test.prefix, test.recursive)
assert.NoError(t, err)
assert.Equal(t, len(test.expectedValue), len(gotk))
for _, key := range gotk {
@ -189,7 +191,7 @@ func TestAzureObjectStorage(t *testing.T) {
t.Run("test useIAM", func(t *testing.T) {
var err error
config.useIAM = true
config.UseIAM = true
_, err = newAzureObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
os.Setenv("AZURE_CLIENT_ID", "00000000-0000-0000-0000-00000000000")
@ -197,16 +199,16 @@ func TestAzureObjectStorage(t *testing.T) {
os.Setenv("AZURE_FEDERATED_TOKEN_FILE", "/var/run/secrets/tokens/azure-identity-token")
_, err = newAzureObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.useIAM = false
config.UseIAM = false
})
t.Run("test key secret", func(t *testing.T) {
var err error
connectionString := os.Getenv("AZURE_STORAGE_CONNECTION_STRING")
os.Setenv("AZURE_STORAGE_CONNECTION_STRING", "")
config.accessKeyID = "devstoreaccount1"
config.secretAccessKeyID = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
config.address = "core.windows.net"
config.AccessKeyID = "devstoreaccount1"
config.SecretAccessKeyID = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
config.Address = "core.windows.net"
_, err = newAzureObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
os.Setenv("AZURE_STORAGE_CONNECTION_STRING", connectionString)
@ -216,11 +218,11 @@ func TestAzureObjectStorage(t *testing.T) {
func TestReadFile(t *testing.T) {
ctx := context.Background()
bucketName := Params.MinioCfg.BucketName.GetValue()
c := &config{
bucketName: bucketName,
createBucket: true,
useIAM: false,
cloudProvider: "azure",
c := &objectstorage.Config{
BucketName: bucketName,
CreateBucket: true,
UseIAM: false,
CloudProvider: "azure",
}
rcm, err := NewRemoteChunkManager(ctx, c)

View File

@ -5,38 +5,39 @@ import (
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
type ChunkManagerFactory struct {
persistentStorage string
config *config
config *objectstorage.Config
}
func NewChunkManagerFactoryWithParam(params *paramtable.ComponentParam) *ChunkManagerFactory {
if params.CommonCfg.StorageType.GetValue() == "local" {
return NewChunkManagerFactory("local", RootPath(params.LocalStorageCfg.Path.GetValue()))
return NewChunkManagerFactory("local", objectstorage.RootPath(params.LocalStorageCfg.Path.GetValue()))
}
return NewChunkManagerFactory(params.CommonCfg.StorageType.GetValue(),
RootPath(params.MinioCfg.RootPath.GetValue()),
Address(params.MinioCfg.Address.GetValue()),
AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
SslCACert(params.MinioCfg.SslCACert.GetValue()),
BucketName(params.MinioCfg.BucketName.GetValue()),
UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
UseVirtualHost(params.MinioCfg.UseVirtualHost.GetAsBool()),
Region(params.MinioCfg.Region.GetValue()),
RequestTimeout(params.MinioCfg.RequestTimeoutMs.GetAsInt64()),
CreateBucket(true),
GcpCredentialJSON(params.MinioCfg.GcpCredentialJSON.GetValue()))
objectstorage.RootPath(params.MinioCfg.RootPath.GetValue()),
objectstorage.Address(params.MinioCfg.Address.GetValue()),
objectstorage.AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
objectstorage.SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
objectstorage.UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
objectstorage.SslCACert(params.MinioCfg.SslCACert.GetValue()),
objectstorage.BucketName(params.MinioCfg.BucketName.GetValue()),
objectstorage.UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
objectstorage.CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
objectstorage.IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
objectstorage.UseVirtualHost(params.MinioCfg.UseVirtualHost.GetAsBool()),
objectstorage.Region(params.MinioCfg.Region.GetValue()),
objectstorage.RequestTimeout(params.MinioCfg.RequestTimeoutMs.GetAsInt64()),
objectstorage.CreateBucket(true),
objectstorage.GcpCredentialJSON(params.MinioCfg.GcpCredentialJSON.GetValue()))
}
func NewChunkManagerFactory(persistentStorage string, opts ...Option) *ChunkManagerFactory {
c := newDefaultConfig()
func NewChunkManagerFactory(persistentStorage string, opts ...objectstorage.Option) *ChunkManagerFactory {
c := objectstorage.NewDefaultConfig()
for _, opt := range opts {
opt(c)
}
@ -49,7 +50,7 @@ func NewChunkManagerFactory(persistentStorage string, opts ...Option) *ChunkMana
func (f *ChunkManagerFactory) newChunkManager(ctx context.Context, engine string) (ChunkManager, error) {
switch engine {
case "local":
return NewLocalChunkManager(RootPath(f.config.rootPath)), nil
return NewLocalChunkManager(objectstorage.RootPath(f.config.RootPath)), nil
case "remote", "minio", "opendal":
return NewRemoteChunkManager(ctx, f.config)
default:

View File

@ -18,78 +18,22 @@ package storage
import (
"context"
"fmt"
"io"
"cloud.google.com/go/storage"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"golang.org/x/oauth2/google"
"google.golang.org/api/googleapi"
"google.golang.org/api/iterator"
"google.golang.org/api/option"
"github.com/milvus-io/milvus/internal/json"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/retry"
)
type GcpNativeObjectStorage struct {
client *storage.Client
}
func newGcpNativeObjectStorageWithConfig(ctx context.Context, c *config) (*GcpNativeObjectStorage, error) {
var client *storage.Client
var err error
var opts []option.ClientOption
var projectId string
if c.address != "" {
completeAddress := "http://"
if c.useSSL {
completeAddress = "https://"
}
completeAddress = completeAddress + c.address + "/storage/v1/"
opts = append(opts, option.WithEndpoint(completeAddress))
}
if c.gcpNativeWithoutAuth {
opts = append(opts, option.WithoutAuthentication())
} else {
creds, err := google.CredentialsFromJSON(ctx, []byte(c.gcpCredentialJSON), storage.ScopeReadWrite)
if err != nil {
return nil, err
}
projectId, err = getProjectId(c.gcpCredentialJSON)
if err != nil {
return nil, err
}
opts = append(opts, option.WithCredentials(creds))
}
client, err = storage.NewClient(ctx, opts...)
if err != nil {
return nil, err
}
if c.bucketName == "" {
return nil, merr.WrapErrParameterInvalidMsg("invalid empty bucket name")
}
// Check bucket validity
checkBucketFn := func() error {
bucket := client.Bucket(c.bucketName)
_, err := bucket.Attrs(ctx)
if err == storage.ErrBucketNotExist && c.createBucket {
log.Info("gcs bucket does not exist, create bucket.", zap.String("bucket name", c.bucketName))
err = client.Bucket(c.bucketName).Create(ctx, projectId, nil)
if err != nil {
return err
}
return nil
}
return err
}
err = retry.Do(ctx, checkBucketFn, retry.Attempts(CheckBucketRetryAttempts))
func newGcpNativeObjectStorageWithConfig(ctx context.Context, c *objectstorage.Config) (*GcpNativeObjectStorage, error) {
client, err := objectstorage.NewGcpObjectStorageClient(ctx, c)
if err != nil {
return nil, err
}
@ -293,19 +237,3 @@ func (gcsReader *GcsReader) Seek(offset int64, whence int) (int64, error) {
gcsReader.position = newOffset
return newOffset, nil
}
func getProjectId(gcpCredentialJSON string) (string, error) {
if gcpCredentialJSON == "" {
return "", errors.New("the JSON string is empty")
}
var data map[string]interface{}
if err := json.Unmarshal([]byte(gcpCredentialJSON), &data); err != nil {
return "", errors.New("failed to parse Google Cloud credentials as JSON")
}
propertyValue, ok := data["project_id"]
projectId := fmt.Sprintf("%v", propertyValue)
if !ok {
return "", errors.New("projectId doesn't exist")
}
return projectId, nil
}

View File

@ -25,27 +25,29 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
)
func TestGcpNativeObjectStorage(t *testing.T) {
ctx := context.Background()
bucketName := "test-bucket"
config := config{
address: "storage.gcs.127.0.0.1.nip.io:4443",
bucketName: bucketName,
createBucket: true,
useIAM: false,
cloudProvider: "gcpnative",
useSSL: false,
gcpNativeWithoutAuth: true,
config := objectstorage.Config{
Address: "storage.gcs.127.0.0.1.nip.io:4443",
BucketName: bucketName,
CreateBucket: true,
UseIAM: false,
CloudProvider: "gcpnative",
UseSSL: false,
GcpNativeWithoutAuth: true,
}
t.Run("test initialize", func(t *testing.T) {
var err error
config.bucketName = ""
config.BucketName = ""
_, err = newGcpNativeObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.bucketName = bucketName
config.BucketName = bucketName
_, err = newGcpNativeObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
})
@ -53,7 +55,7 @@ func TestGcpNativeObjectStorage(t *testing.T) {
t.Run("test load", func(t *testing.T) {
testCM, err := newGcpNativeObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
defer testCM.DeleteBucket(ctx, config.bucketName)
defer testCM.DeleteBucket(ctx, config.BucketName)
prepareTests := []struct {
key string
@ -67,7 +69,7 @@ func TestGcpNativeObjectStorage(t *testing.T) {
}
for _, test := range prepareTests {
err := testCM.PutObject(ctx, config.bucketName, test.key, bytes.NewReader(test.value),
err := testCM.PutObject(ctx, config.BucketName, test.key, bytes.NewReader(test.value),
int64(len(test.value)))
require.NoError(t, err)
}
@ -91,19 +93,19 @@ func TestGcpNativeObjectStorage(t *testing.T) {
for _, test := range loadTests {
t.Run(test.description, func(t *testing.T) {
if test.isvalid {
got, err := testCM.GetObject(ctx, config.bucketName, test.loadKey, 0, 1024)
got, err := testCM.GetObject(ctx, config.BucketName, test.loadKey, 0, 1024)
assert.NoError(t, err)
contentData, err := io.ReadAll(got)
assert.NoError(t, err)
assert.Equal(t, len(contentData), len(test.expectedValue))
assert.Equal(t, test.expectedValue, contentData)
statSize, err := testCM.StatObject(ctx, config.bucketName, test.loadKey)
statSize, err := testCM.StatObject(ctx, config.BucketName, test.loadKey)
assert.NoError(t, err)
assert.Equal(t, statSize, int64(len(contentData)))
_, err = testCM.GetObject(ctx, config.bucketName, test.loadKey, 1, 1023)
_, err = testCM.GetObject(ctx, config.BucketName, test.loadKey, 1, 1023)
assert.NoError(t, err)
} else {
got, err := testCM.GetObject(ctx, config.bucketName, test.loadKey, 0, 1024)
got, err := testCM.GetObject(ctx, config.BucketName, test.loadKey, 0, 1024)
assert.Error(t, err)
assert.Empty(t, got)
}
@ -124,12 +126,12 @@ func TestGcpNativeObjectStorage(t *testing.T) {
for _, test := range loadWithPrefixTests {
t.Run(test.description, func(t *testing.T) {
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.bucketName,
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.BucketName,
test.prefix, false)
assert.NoError(t, err)
assert.Equal(t, len(test.expectedValue), len(gotk))
for _, key := range gotk {
err := testCM.RemoveObject(ctx, config.bucketName, key)
err := testCM.RemoveObject(ctx, config.BucketName, key)
assert.NoError(t, err)
}
})
@ -139,7 +141,7 @@ func TestGcpNativeObjectStorage(t *testing.T) {
t.Run("test list", func(t *testing.T) {
testCM, err := newGcpNativeObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
defer testCM.DeleteBucket(ctx, config.bucketName)
defer testCM.DeleteBucket(ctx, config.BucketName)
prepareTests := []struct {
valid bool
@ -155,11 +157,11 @@ func TestGcpNativeObjectStorage(t *testing.T) {
}
for _, test := range prepareTests {
err := testCM.PutObject(ctx, config.bucketName, test.key, bytes.NewReader(test.value),
err := testCM.PutObject(ctx, config.BucketName, test.key, bytes.NewReader(test.value),
int64(len(test.value)))
require.Nil(t, err)
if !test.valid {
err := testCM.RemoveObject(ctx, config.bucketName, test.key)
err := testCM.RemoveObject(ctx, config.BucketName, test.key)
require.Nil(t, err)
}
}
@ -177,7 +179,7 @@ func TestGcpNativeObjectStorage(t *testing.T) {
for _, test := range insertWithPrefixTests {
t.Run(fmt.Sprintf("prefix: %s, recursive: %t", test.prefix, test.recursive), func(t *testing.T) {
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.bucketName,
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.BucketName,
test.prefix, test.recursive)
assert.NoError(t, err)
assert.Equal(t, len(test.expectedValue), len(gotk))
@ -192,14 +194,14 @@ func TestGcpNativeObjectStorage(t *testing.T) {
func TestGcpNativeReadFile(t *testing.T) {
ctx := context.Background()
bucketName := "test-bucket"
c := &config{
address: "storage.gcs.127.0.0.1.nip.io:4443",
bucketName: bucketName,
createBucket: true,
useIAM: false,
cloudProvider: "gcpnative",
useSSL: false,
gcpNativeWithoutAuth: true,
c := &objectstorage.Config{
Address: "storage.gcs.127.0.0.1.nip.io:4443",
BucketName: bucketName,
CreateBucket: true,
UseIAM: false,
CloudProvider: "gcpnative",
UseSSL: false,
GcpNativeWithoutAuth: true,
}
rcm, err := NewRemoteChunkManager(ctx, c)

View File

@ -29,6 +29,7 @@ import (
"golang.org/x/exp/mmap"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
)
@ -40,13 +41,13 @@ type LocalChunkManager struct {
var _ ChunkManager = (*LocalChunkManager)(nil)
// NewLocalChunkManager create a new local manager object.
func NewLocalChunkManager(opts ...Option) *LocalChunkManager {
c := newDefaultConfig()
func NewLocalChunkManager(opts ...objectstorage.Option) *LocalChunkManager {
c := objectstorage.NewDefaultConfig()
for _, opt := range opts {
opt(c)
}
return &LocalChunkManager{
localPath: c.rootPath,
localPath: c.RootPath,
}
}

View File

@ -24,6 +24,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
)
const (
@ -34,14 +36,14 @@ func TestLocalCM(t *testing.T) {
ctx := context.Background()
t.Run("test RootPath", func(t *testing.T) {
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
assert.Equal(t, localPath, testCM.RootPath())
})
t.Run("test load", func(t *testing.T) {
testLoadRoot := "test_load"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
prepareTests := []struct {
@ -150,7 +152,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test write", func(t *testing.T) {
testMultiSaveRoot := "test_write"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
key1 := path.Join(localPath, testMultiSaveRoot, "key_1")
@ -177,7 +179,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test MultiSave", func(t *testing.T) {
testMultiSaveRoot := "test_multisave"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
err := testCM.Write(ctx, path.Join(localPath, testMultiSaveRoot, "key_1"), []byte("111"))
@ -207,7 +209,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test Remove", func(t *testing.T) {
testRemoveRoot := "test_remove"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
// empty prefix is not allowed
err := testCM.RemoveWithPrefix(ctx, "")
@ -308,7 +310,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test ReadAt", func(t *testing.T) {
testLoadPartialRoot := "read_at"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
key := path.Join(localPath, testLoadPartialRoot, "TestMinIOKV_LoadPartial_key")
@ -353,7 +355,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test Size", func(t *testing.T) {
testGetSizeRoot := "get_size"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
key := path.Join(localPath, testGetSizeRoot, "TestMinIOKV_GetSize_key")
@ -376,7 +378,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test read", func(t *testing.T) {
testGetSizeRoot := "get_path"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
key := path.Join(localPath, testGetSizeRoot, "TestMinIOKV_GetPath_key")
@ -397,7 +399,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test Path", func(t *testing.T) {
testGetSizeRoot := "get_path"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
key := path.Join(localPath, testGetSizeRoot, "TestMinIOKV_GetPath_key")
@ -420,7 +422,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test Prefix", func(t *testing.T) {
testPrefix := "prefix"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
// write 2 files:
@ -465,7 +467,7 @@ func TestLocalCM(t *testing.T) {
t.Run("test ListWithPrefix", func(t *testing.T) {
testPrefix := "prefix-ListWithPrefix"
testCM := NewLocalChunkManager(RootPath(localPath))
testCM := NewLocalChunkManager(objectstorage.RootPath(localPath))
defer testCM.RemoveWithPrefix(ctx, testCM.RootPath())
// write 4 files:

View File

@ -18,150 +18,24 @@ package storage
import (
"context"
"fmt"
"io"
"os"
"strings"
"github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/storage/aliyun"
"github.com/milvus-io/milvus/internal/storage/gcp"
"github.com/milvus-io/milvus/internal/storage/tencent"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/retry"
)
var CheckBucketRetryAttempts uint = 20
var _ ObjectStorage = (*MinioObjectStorage)(nil)
type MinioObjectStorage struct {
*minio.Client
}
func newMinioClient(ctx context.Context, c *config) (*minio.Client, error) {
var creds *credentials.Credentials
newMinioFn := minio.New
bucketLookupType := minio.BucketLookupAuto
if c.useVirtualHost {
bucketLookupType = minio.BucketLookupDNS
}
matchedDefault := false
switch c.cloudProvider {
case CloudProviderAliyun:
// auto doesn't work for aliyun, so we set to dns deliberately
bucketLookupType = minio.BucketLookupDNS
if c.useIAM {
newMinioFn = aliyun.NewMinioClient
} else {
creds = credentials.NewStaticV4(c.accessKeyID, c.secretAccessKeyID, "")
}
case CloudProviderGCP:
newMinioFn = gcp.NewMinioClient
if !c.useIAM {
creds = credentials.NewStaticV2(c.accessKeyID, c.secretAccessKeyID, "")
}
case CloudProviderTencent:
bucketLookupType = minio.BucketLookupDNS
newMinioFn = tencent.NewMinioClient
if !c.useIAM {
creds = credentials.NewStaticV4(c.accessKeyID, c.secretAccessKeyID, "")
}
default: // aws, minio
matchedDefault = true
}
// Compatibility logic. If the cloud provider is not specified in the request,
// it shall be inferred based on the service address.
if matchedDefault {
matchedDefault = false
switch {
case strings.Contains(c.address, gcp.GcsDefaultAddress):
newMinioFn = gcp.NewMinioClient
if !c.useIAM {
creds = credentials.NewStaticV2(c.accessKeyID, c.secretAccessKeyID, "")
}
case strings.Contains(c.address, aliyun.OSSAddressFeatureString):
// auto doesn't work for aliyun, so we set to dns deliberately
bucketLookupType = minio.BucketLookupDNS
if c.useIAM {
newMinioFn = aliyun.NewMinioClient
} else {
creds = credentials.NewStaticV4(c.accessKeyID, c.secretAccessKeyID, "")
}
default:
matchedDefault = true
}
}
if matchedDefault {
// aws, minio
if c.useIAM {
creds = credentials.NewIAM("")
} else {
creds = credentials.NewStaticV4(c.accessKeyID, c.secretAccessKeyID, "")
}
}
// We must set the cert path by os environment variable "SSL_CERT_FILE",
// because the minio.DefaultTransport() need this path to read the file content,
// we shouldn't read this file by ourself.
if c.useSSL && len(c.sslCACert) > 0 {
err := os.Setenv("SSL_CERT_FILE", c.sslCACert)
if err != nil {
return nil, err
}
}
minioOpts := &minio.Options{
BucketLookup: bucketLookupType,
Creds: creds,
Secure: c.useSSL,
Region: c.region,
}
minIOClient, err := newMinioFn(c.address, minioOpts)
// options nil or invalid formatted endpoint, don't need to retry
if err != nil {
return nil, err
}
var bucketExists bool
// check valid in first query
checkBucketFn := func() error {
bucketExists, err = minIOClient.BucketExists(ctx, c.bucketName)
if err != nil {
log.Warn("failed to check blob bucket exist", zap.String("bucket", c.bucketName), zap.Error(err))
return err
}
if !bucketExists {
if c.createBucket {
log.Info("blob bucket not exist, create bucket.", zap.String("bucket name", c.bucketName))
err := minIOClient.MakeBucket(ctx, c.bucketName, minio.MakeBucketOptions{})
if err != nil {
log.Warn("failed to create blob bucket", zap.String("bucket", c.bucketName), zap.Error(err))
return err
}
} else {
return fmt.Errorf("bucket %s not Existed", c.bucketName)
}
}
return nil
}
err = retry.Do(ctx, checkBucketFn, retry.Attempts(CheckBucketRetryAttempts))
if err != nil {
return nil, err
}
return minIOClient, nil
}
func newMinioObjectStorageWithConfig(ctx context.Context, c *config) (*MinioObjectStorage, error) {
minIOClient, err := newMinioClient(ctx, c)
func newMinioObjectStorageWithConfig(ctx context.Context, c *objectstorage.Config) (*MinioObjectStorage, error) {
minIOClient, err := objectstorage.NewMinioClient(ctx, c)
if err != nil {
return nil, err
}

View File

@ -27,29 +27,31 @@ import (
"github.com/minio/minio-go/v7"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
)
func TestMinioObjectStorage(t *testing.T) {
ctx := context.Background()
config := config{
address: Params.MinioCfg.Address.GetValue(),
accessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
secretAccessKeyID: Params.MinioCfg.SecretAccessKey.GetValue(),
rootPath: Params.MinioCfg.RootPath.GetValue(),
config := objectstorage.Config{
Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKeyID: Params.MinioCfg.SecretAccessKey.GetValue(),
RootPath: Params.MinioCfg.RootPath.GetValue(),
bucketName: Params.MinioCfg.BucketName.GetValue(),
createBucket: true,
useIAM: false,
cloudProvider: "minio",
BucketName: Params.MinioCfg.BucketName.GetValue(),
CreateBucket: true,
UseIAM: false,
CloudProvider: "minio",
}
t.Run("test initialize", func(t *testing.T) {
var err error
bucketName := config.bucketName
config.bucketName = ""
bucketName := config.BucketName
config.BucketName = ""
_, err = newMinioObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.bucketName = bucketName
config.BucketName = bucketName
_, err = newMinioObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
})
@ -57,7 +59,7 @@ func TestMinioObjectStorage(t *testing.T) {
t.Run("test load", func(t *testing.T) {
testCM, err := newMinioObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
defer testCM.RemoveBucket(ctx, config.bucketName)
defer testCM.RemoveBucket(ctx, config.BucketName)
prepareTests := []struct {
key string
@ -71,7 +73,7 @@ func TestMinioObjectStorage(t *testing.T) {
}
for _, test := range prepareTests {
err := testCM.PutObject(ctx, config.bucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
err := testCM.PutObject(ctx, config.BucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
require.NoError(t, err)
}
@ -94,19 +96,19 @@ func TestMinioObjectStorage(t *testing.T) {
for _, test := range loadTests {
t.Run(test.description, func(t *testing.T) {
if test.isvalid {
got, err := testCM.GetObject(ctx, config.bucketName, test.loadKey, 0, 1024)
got, err := testCM.GetObject(ctx, config.BucketName, test.loadKey, 0, 1024)
assert.NoError(t, err)
contentData, err := io.ReadAll(got)
assert.NoError(t, err)
assert.Equal(t, len(contentData), len(test.expectedValue))
assert.Equal(t, test.expectedValue, contentData)
statSize, err := testCM.StatObject(ctx, config.bucketName, test.loadKey)
statSize, err := testCM.StatObject(ctx, config.BucketName, test.loadKey)
assert.NoError(t, err)
assert.Equal(t, statSize, int64(len(contentData)))
_, err = testCM.GetObject(ctx, config.bucketName, test.loadKey, 1, 1023)
_, err = testCM.GetObject(ctx, config.BucketName, test.loadKey, 1, 1023)
assert.NoError(t, err)
} else {
got, err := testCM.GetObject(ctx, config.bucketName, test.loadKey, 0, 1024)
got, err := testCM.GetObject(ctx, config.BucketName, test.loadKey, 0, 1024)
assert.NoError(t, err)
_, err = io.ReadAll(got)
errResponse := minio.ToErrorResponse(err)
@ -133,11 +135,11 @@ func TestMinioObjectStorage(t *testing.T) {
for _, test := range loadWithPrefixTests {
t.Run(test.description, func(t *testing.T) {
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.bucketName, test.prefix, false)
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.BucketName, test.prefix, false)
assert.NoError(t, err)
assert.Equal(t, len(test.expectedValue), len(gotk))
for _, key := range gotk {
err := testCM.RemoveObject(ctx, config.bucketName, key)
err := testCM.RemoveObject(ctx, config.BucketName, key)
assert.NoError(t, err)
}
})
@ -147,7 +149,7 @@ func TestMinioObjectStorage(t *testing.T) {
t.Run("test list", func(t *testing.T) {
testCM, err := newMinioObjectStorageWithConfig(ctx, &config)
assert.Equal(t, err, nil)
defer testCM.RemoveBucketWithOptions(ctx, config.bucketName, minio.RemoveBucketOptions{
defer testCM.RemoveBucketWithOptions(ctx, config.BucketName, minio.RemoveBucketOptions{
ForceDelete: true,
})
@ -168,7 +170,7 @@ func TestMinioObjectStorage(t *testing.T) {
for _, test := range prepareTests {
t.Run(test.key, func(t *testing.T) {
err := testCM.PutObject(ctx, config.bucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
err := testCM.PutObject(ctx, config.BucketName, test.key, bytes.NewReader(test.value), int64(len(test.value)))
require.Equal(t, test.valid, err == nil, err)
})
}
@ -186,7 +188,7 @@ func TestMinioObjectStorage(t *testing.T) {
for _, test := range insertWithPrefixTests {
t.Run(fmt.Sprintf("prefix: %s, recursive: %t", test.prefix, test.recursive), func(t *testing.T) {
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.bucketName, test.prefix, test.recursive)
gotk, _, err := listAllObjectsWithPrefixAtBucket(ctx, testCM, config.BucketName, test.prefix, test.recursive)
assert.NoError(t, err)
assert.Equal(t, len(test.expectedValue), len(gotk))
for _, key := range gotk {
@ -198,35 +200,35 @@ func TestMinioObjectStorage(t *testing.T) {
t.Run("test useIAM", func(t *testing.T) {
var err error
config.useIAM = true
config.UseIAM = true
_, err = newMinioObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.useIAM = false
config.UseIAM = false
})
t.Run("test ssl", func(t *testing.T) {
var err error
config.useSSL = true
config.sslCACert = "/tmp/dummy.crt"
config.UseSSL = true
config.SslCACert = "/tmp/dummy.crt"
_, err = newMinioObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.useSSL = false
config.UseSSL = false
})
t.Run("test cloud provider", func(t *testing.T) {
var err error
cloudProvider := config.cloudProvider
config.cloudProvider = "aliyun"
config.useIAM = true
cloudProvider := config.CloudProvider
config.CloudProvider = "aliyun"
config.UseIAM = true
_, err = newMinioObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.useIAM = false
config.UseIAM = false
_, err = newMinioObjectStorageWithConfig(ctx, &config)
assert.Error(t, err)
config.cloudProvider = "gcp"
config.CloudProvider = "gcp"
_, err = newMinioObjectStorageWithConfig(ctx, &config)
assert.NoError(t, err)
config.cloudProvider = cloudProvider
config.CloudProvider = cloudProvider
})
}

View File

@ -1,118 +0,0 @@
package storage
// Option for setting params used by chunk manager client.
type config struct {
address string
bucketName string
accessKeyID string
secretAccessKeyID string
useSSL bool
sslCACert string
createBucket bool
rootPath string
useIAM bool
cloudProvider string
iamEndpoint string
useVirtualHost bool
region string
requestTimeoutMs int64
gcpCredentialJSON string
gcpNativeWithoutAuth bool // used for Unit Testing
}
func newDefaultConfig() *config {
return &config{}
}
// Option is used to config the retry function.
type Option func(*config)
func Address(addr string) Option {
return func(c *config) {
c.address = addr
}
}
func BucketName(bucketName string) Option {
return func(c *config) {
c.bucketName = bucketName
}
}
func AccessKeyID(accessKeyID string) Option {
return func(c *config) {
c.accessKeyID = accessKeyID
}
}
func SecretAccessKeyID(secretAccessKeyID string) Option {
return func(c *config) {
c.secretAccessKeyID = secretAccessKeyID
}
}
func UseSSL(useSSL bool) Option {
return func(c *config) {
c.useSSL = useSSL
}
}
func SslCACert(sslCACert string) Option {
return func(c *config) {
c.sslCACert = sslCACert
}
}
func CreateBucket(createBucket bool) Option {
return func(c *config) {
c.createBucket = createBucket
}
}
func RootPath(rootPath string) Option {
return func(c *config) {
c.rootPath = rootPath
}
}
func UseIAM(useIAM bool) Option {
return func(c *config) {
c.useIAM = useIAM
}
}
func CloudProvider(cloudProvider string) Option {
return func(c *config) {
c.cloudProvider = cloudProvider
}
}
func IAMEndpoint(iamEndpoint string) Option {
return func(c *config) {
c.iamEndpoint = iamEndpoint
}
}
func UseVirtualHost(useVirtualHost bool) Option {
return func(c *config) {
c.useVirtualHost = useVirtualHost
}
}
func Region(region string) Option {
return func(c *config) {
c.region = region
}
}
func RequestTimeout(requestTimeoutMs int64) Option {
return func(c *config) {
c.requestTimeoutMs = requestTimeoutMs
}
}
func GcpCredentialJSON(gcpCredentialJSON string) Option {
return func(c *config) {
c.gcpCredentialJSON = gcpCredentialJSON
}
}

View File

@ -26,7 +26,7 @@ import (
"github.com/Azure/azure-sdk-for-go/sdk/azcore"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/bloberror"
"github.com/cockroachdb/errors"
minio "github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7"
"go.uber.org/zap"
"golang.org/x/exp/mmap"
"golang.org/x/sync/errgroup"
@ -34,20 +34,12 @@ import (
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/metrics"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/retry"
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
)
const (
CloudProviderGCP = "gcp"
CloudProviderGCPNative = "gcpnative"
CloudProviderAWS = "aws"
CloudProviderAliyun = "aliyun"
CloudProviderAzure = "azure"
CloudProviderTencent = "tencent"
)
// ChunkObjectWalkFunc is the callback function for walking objects.
// If return false, WalkWithObjects will stop.
// Otherwise, WalkWithObjects will continue until reach the last object.
@ -65,7 +57,7 @@ type ObjectStorage interface {
RemoveObject(ctx context.Context, bucketName, objectName string) error
}
// RemoteChunkManager is responsible for read and write data stored in minio.
// RemoteChunkManager is responsible for read and write data stored in mminio.
type RemoteChunkManager struct {
client ObjectStorage
@ -76,12 +68,12 @@ type RemoteChunkManager struct {
var _ ChunkManager = (*RemoteChunkManager)(nil)
func NewRemoteChunkManager(ctx context.Context, c *config) (*RemoteChunkManager, error) {
func NewRemoteChunkManager(ctx context.Context, c *objectstorage.Config) (*RemoteChunkManager, error) {
var client ObjectStorage
var err error
if c.cloudProvider == CloudProviderAzure {
if c.CloudProvider == objectstorage.CloudProviderAzure {
client, err = newAzureObjectStorageWithConfig(ctx, c)
} else if c.cloudProvider == CloudProviderGCPNative {
} else if c.CloudProvider == objectstorage.CloudProviderGCPNative {
client, err = newGcpNativeObjectStorageWithConfig(ctx, c)
} else {
client, err = newMinioObjectStorageWithConfig(ctx, c)
@ -91,10 +83,10 @@ func NewRemoteChunkManager(ctx context.Context, c *config) (*RemoteChunkManager,
}
mcm := &RemoteChunkManager{
client: client,
bucketName: c.bucketName,
rootPath: strings.TrimLeft(c.rootPath, "/"),
bucketName: c.BucketName,
rootPath: strings.TrimLeft(c.RootPath, "/"),
}
log.Info("remote chunk manager init success.", zap.String("remote", c.cloudProvider), zap.String("bucketname", c.bucketName), zap.String("root", mcm.RootPath()))
log.Info("remote chunk manager init success.", zap.String("remote", c.CloudProvider), zap.String("bucketname", c.BucketName), zap.String("root", mcm.RootPath()))
return mcm, nil
}

View File

@ -25,6 +25,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
)
@ -39,27 +40,27 @@ func newAzureChunkManager(ctx context.Context, bucketName string, rootPath strin
func newRemoteChunkManager(ctx context.Context, cloudProvider string, bucketName string, rootPath string) (ChunkManager, error) {
factory := NewChunkManagerFactory("remote",
RootPath(rootPath),
Address(Params.MinioCfg.Address.GetValue()),
AccessKeyID(Params.MinioCfg.AccessKeyID.GetValue()),
SecretAccessKeyID(Params.MinioCfg.SecretAccessKey.GetValue()),
UseSSL(Params.MinioCfg.UseSSL.GetAsBool()),
SslCACert(Params.MinioCfg.SslCACert.GetValue()),
BucketName(bucketName),
UseIAM(Params.MinioCfg.UseIAM.GetAsBool()),
CloudProvider(cloudProvider),
IAMEndpoint(Params.MinioCfg.IAMEndpoint.GetValue()),
CreateBucket(true))
objectstorage.RootPath(rootPath),
objectstorage.Address(Params.MinioCfg.Address.GetValue()),
objectstorage.AccessKeyID(Params.MinioCfg.AccessKeyID.GetValue()),
objectstorage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey.GetValue()),
objectstorage.UseSSL(Params.MinioCfg.UseSSL.GetAsBool()),
objectstorage.SslCACert(Params.MinioCfg.SslCACert.GetValue()),
objectstorage.BucketName(bucketName),
objectstorage.UseIAM(Params.MinioCfg.UseIAM.GetAsBool()),
objectstorage.CloudProvider(cloudProvider),
objectstorage.IAMEndpoint(Params.MinioCfg.IAMEndpoint.GetValue()),
objectstorage.CreateBucket(true))
return factory.NewPersistentStorageChunkManager(ctx)
}
func TestInitRemoteChunkManager(t *testing.T) {
ctx := context.Background()
client, err := NewRemoteChunkManager(ctx, &config{
bucketName: Params.MinioCfg.BucketName.GetValue(),
createBucket: true,
useIAM: false,
cloudProvider: "azure",
client, err := NewRemoteChunkManager(ctx, &objectstorage.Config{
BucketName: Params.MinioCfg.BucketName.GetValue(),
CreateBucket: true,
UseIAM: false,
CloudProvider: "azure",
})
assert.NoError(t, err)
assert.NotNil(t, client)

View File

@ -38,6 +38,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
@ -1364,16 +1365,16 @@ func Min(a, b int64) int64 {
func NewTestChunkManagerFactory(params *paramtable.ComponentParam, rootPath string) *ChunkManagerFactory {
return NewChunkManagerFactory("minio",
RootPath(rootPath),
Address(params.MinioCfg.Address.GetValue()),
AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
BucketName(params.MinioCfg.BucketName.GetValue()),
UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
CreateBucket(true))
objectstorage.RootPath(rootPath),
objectstorage.Address(params.MinioCfg.Address.GetValue()),
objectstorage.AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
objectstorage.SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
objectstorage.UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
objectstorage.BucketName(params.MinioCfg.BucketName.GetValue()),
objectstorage.UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
objectstorage.CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
objectstorage.IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
objectstorage.CreateBucket(true))
}
func GetFilesSize(ctx context.Context, paths []string, cm ChunkManager) (int64, error) {

View File

@ -11,6 +11,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/metrics"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
@ -42,7 +43,7 @@ func NewDefaultFactory(standAlone bool) *DefaultFactory {
standAlone: standAlone,
msgStreamFactory: msgstream.NewRocksmqFactory("/tmp/milvus/rocksmq/", &paramtable.Get().ServiceParam),
chunkManagerFactory: storage.NewChunkManagerFactory("local",
storage.RootPath("/tmp/milvus")),
objectstorage.RootPath("/tmp/milvus")),
}
}

View File

@ -31,6 +31,7 @@ import (
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/testutil"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
@ -140,7 +141,7 @@ func (suite *ReaderSuite) run(dataType schemapb.DataType, elemType schemapb.Data
// read from csv file
ctx := context.Background()
f := storage.NewChunkManagerFactory("local", storage.RootPath("/tmp/milvus_test/test_csv_reader/"))
f := storage.NewChunkManagerFactory("local", objectstorage.RootPath("/tmp/milvus_test/test_csv_reader/"))
cm, err := f.NewPersistentStorageChunkManager(ctx)
suite.NoError(err)

View File

@ -37,6 +37,7 @@ import (
"github.com/milvus-io/milvus/internal/util/nullutil"
"github.com/milvus-io/milvus/internal/util/testutil"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/objectstorage"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
@ -174,7 +175,7 @@ func (s *ReaderSuite) run(dataType schemapb.DataType, elemType schemapb.DataType
assert.NoError(s.T(), err)
ctx := context.Background()
f := storage.NewChunkManagerFactory("local", storage.RootPath("/tmp/milvus_test/test_parquet_reader/"))
f := storage.NewChunkManagerFactory("local", objectstorage.RootPath("/tmp/milvus_test/test_parquet_reader/"))
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(s.T(), err)
reader, err := NewReader(ctx, cm, schema, filePath, 64*1024*1024)
@ -276,7 +277,7 @@ func (s *ReaderSuite) failRun(dt schemapb.DataType, isDynamic bool) {
assert.NoError(s.T(), err)
ctx := context.Background()
f := storage.NewChunkManagerFactory("local", storage.RootPath("/tmp/milvus_test/test_parquet_reader/"))
f := storage.NewChunkManagerFactory("local", objectstorage.RootPath("/tmp/milvus_test/test_parquet_reader/"))
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(s.T(), err)
reader, err := NewReader(ctx, cm, schema, filePath, 64*1024*1024)
@ -329,7 +330,7 @@ func (s *ReaderSuite) runWithDefaultValue(dataType schemapb.DataType, elemType s
assert.NoError(s.T(), err)
ctx := context.Background()
f := storage.NewChunkManagerFactory("local", storage.RootPath("/tmp/milvus_test/test_parquet_reader/"))
f := storage.NewChunkManagerFactory("local", objectstorage.RootPath("/tmp/milvus_test/test_parquet_reader/"))
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(s.T(), err)
schema.Fields[2].Nullable = nullable

View File

@ -3,6 +3,11 @@ module github.com/milvus-io/milvus/pkg/v2
go 1.21
require (
cloud.google.com/go/storage v1.43.0
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0
github.com/aliyun/credentials-go v1.2.7
github.com/apache/pulsar-client-go v0.6.1-0.20210728062540-29414db801a7
github.com/benesch/cgosymbolizer v0.0.0-20190515212042-bec6fe6e597b
github.com/blang/semver/v4 v4.0.0
@ -14,8 +19,9 @@ require (
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
github.com/jolestar/go-commons-pool/v2 v2.1.2
github.com/json-iterator/go v1.1.12
github.com/klauspost/compress v1.17.7
github.com/klauspost/compress v1.17.9
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250225103150-0a1988183e53
github.com/minio/minio-go/v7 v7.0.73
github.com/nats-io/nats-server/v2 v2.10.12
github.com/nats-io/nats.go v1.34.1
github.com/panjf2000/ants/v2 v2.7.2
@ -31,6 +37,7 @@ require (
github.com/streamnative/pulsarctl v0.5.0
github.com/stretchr/testify v1.9.0
github.com/tecbot/gorocksdb v0.0.0-20191217155057-f0fad39f321c
github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.865
github.com/tidwall/gjson v1.17.0
github.com/tikv/client-go/v2 v2.0.4
github.com/uber/jaeger-client-go v2.30.0+incompatible
@ -52,8 +59,10 @@ require (
golang.org/x/crypto v0.31.0
golang.org/x/exp v0.0.0-20230224173230-c95f2b4c22f2
golang.org/x/net v0.33.0
golang.org/x/oauth2 v0.21.0
golang.org/x/sync v0.10.0
golang.org/x/sys v0.28.0
google.golang.org/api v0.187.0
google.golang.org/grpc v1.65.0
google.golang.org/protobuf v1.34.2
gopkg.in/natefinch/lumberjack.v2 v2.0.0
@ -62,11 +71,20 @@ require (
)
require (
cloud.google.com/go v0.115.0 // indirect
cloud.google.com/go/auth v0.6.1 // indirect
cloud.google.com/go/auth/oauth2adapt v0.2.2 // indirect
cloud.google.com/go/compute/metadata v0.3.0 // indirect
cloud.google.com/go/iam v1.1.8 // indirect
github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect
github.com/99designs/keyring v1.2.1 // indirect
github.com/AthenZ/athenz v1.10.39 // indirect
github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 // indirect
github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect
github.com/BurntSushi/toml v1.2.1 // indirect
github.com/DataDog/zstd v1.5.0 // indirect
github.com/alibabacloud-go/debug v0.0.0-20190504072949-9472017b5c68 // indirect
github.com/alibabacloud-go/tea v1.1.8 // indirect
github.com/ardielle/ardielle-go v1.5.2 // indirect
github.com/beorn7/perks v1.0.1 // indirect
github.com/bits-and-blooms/bitset v1.4.0 // indirect
@ -81,24 +99,32 @@ require (
github.com/davecgh/go-spew v1.1.1 // indirect
github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect
github.com/docker/go-units v0.4.0 // indirect
github.com/dustin/go-humanize v1.0.0 // indirect
github.com/dustin/go-humanize v1.0.1 // indirect
github.com/dvsekhvalnov/jose2go v1.6.0 // indirect
github.com/facebookgo/ensure v0.0.0-20200202191622-63f1cf65ac4c // indirect
github.com/facebookgo/stack v0.0.0-20160209184415-751773369052 // indirect
github.com/facebookgo/subset v0.0.0-20200203212716-c811ad88dec4 // indirect
github.com/felixge/httpsnoop v1.0.4 // indirect
github.com/form3tech-oss/jwt-go v3.2.3+incompatible // indirect
github.com/getsentry/sentry-go v0.12.0 // indirect
github.com/go-ini/ini v1.67.0 // indirect
github.com/go-logr/logr v1.4.2 // indirect
github.com/go-logr/stdr v1.2.2 // indirect
github.com/go-ole/go-ole v1.2.6 // indirect
github.com/goccy/go-json v0.10.3 // indirect
github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect
github.com/godbus/dbus/v5 v5.0.4 // indirect
github.com/gogo/protobuf v1.3.2 // indirect
github.com/golang-jwt/jwt v3.2.2+incompatible // indirect
github.com/golang-jwt/jwt/v5 v5.2.1 // indirect
github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect
github.com/golang/protobuf v1.5.4 // indirect
github.com/golang/snappy v0.0.4 // indirect
github.com/google/btree v1.1.2 // indirect
github.com/google/s2a-go v0.1.7 // indirect
github.com/google/uuid v1.6.0 // indirect
github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect
github.com/googleapis/gax-go/v2 v2.12.5 // indirect
github.com/gorilla/websocket v1.4.2 // indirect
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect
github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect
@ -108,14 +134,17 @@ require (
github.com/hashicorp/go-multierror v1.1.1 // indirect
github.com/ianlancetaylor/cgosymbolizer v0.0.0-20221217025313-27d3c9f66b6a // indirect
github.com/jonboulle/clockwork v0.2.2 // indirect
github.com/klauspost/cpuid/v2 v2.2.8 // indirect
github.com/kr/pretty v0.3.1 // indirect
github.com/kr/text v0.2.0 // indirect
github.com/kylelemons/godebug v1.1.0 // indirect
github.com/linkedin/goavro/v2 v2.11.1 // indirect
github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect
github.com/mattn/go-isatty v0.0.19 // indirect
github.com/mattn/go-runewidth v0.0.8 // indirect
github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect
github.com/minio/highwayhash v1.0.2 // indirect
github.com/minio/md5-simd v1.1.2 // indirect
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect
github.com/modern-go/reflect2 v1.0.2 // indirect
github.com/mtibben/percent v0.2.1 // indirect
@ -131,6 +160,7 @@ require (
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect
github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a // indirect
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 // indirect
github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect
github.com/pkg/errors v0.9.1 // indirect
github.com/pmezard/go-difflib v1.0.0 // indirect
github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect
@ -138,7 +168,8 @@ require (
github.com/prometheus/common v0.42.0 // indirect
github.com/prometheus/procfs v0.9.0 // indirect
github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect
github.com/rogpeppe/go-internal v1.10.0 // indirect
github.com/rogpeppe/go-internal v1.12.0 // indirect
github.com/rs/xid v1.5.0 // indirect
github.com/soheilhy/cmux v0.1.5 // indirect
github.com/spf13/pflag v1.0.5 // indirect
github.com/stathat/consistent v1.0.0 // indirect
@ -158,19 +189,21 @@ require (
go.etcd.io/etcd/client/v2 v2.305.5 // indirect
go.etcd.io/etcd/pkg/v3 v3.5.5 // indirect
go.etcd.io/etcd/raft/v3 v3.5.5 // indirect
go.opencensus.io v0.24.0 // indirect
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 // indirect
go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.20.0 // indirect
go.opentelemetry.io/otel/metric v1.28.0 // indirect
go.opentelemetry.io/proto/otlp v1.0.0 // indirect
go.uber.org/multierr v1.11.0 // indirect
golang.org/x/mod v0.17.0 // indirect
golang.org/x/oauth2 v0.20.0 // indirect
golang.org/x/term v0.27.0 // indirect
golang.org/x/text v0.21.0 // indirect
golang.org/x/time v0.5.0 // indirect
google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 // indirect
google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157 // indirect
google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d // indirect
google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4 // indirect
google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf // indirect
gopkg.in/inf.v0 v0.9.1 // indirect
gopkg.in/ini.v1 v1.56.0 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
sigs.k8s.io/yaml v1.3.0 // indirect
)

View File

@ -18,15 +18,27 @@ cloud.google.com/go v0.74.0/go.mod h1:VV1xSbzvo+9QJOxLDaJfTjx5e+MePCpCWwvftOeQmW
cloud.google.com/go v0.78.0/go.mod h1:QjdrLG0uq+YwhjoVOLsS1t7TW8fs36kLs4XO5R5ECHg=
cloud.google.com/go v0.79.0/go.mod h1:3bzgcEeQlzbuEAYu4mrWhKqWjmpprinYgKJLgKHnbb8=
cloud.google.com/go v0.81.0/go.mod h1:mk/AM35KwGk/Nm2YSeZbxXdrNK3KZOYHmLkOqC2V6E0=
cloud.google.com/go v0.115.0 h1:CnFSK6Xo3lDYRoBKEcAtia6VSC837/ZkJuRduSFnr14=
cloud.google.com/go v0.115.0/go.mod h1:8jIM5vVgoAEoiVxQ/O4BFTfHqulPZgs/ufEzMcFMdWU=
cloud.google.com/go/auth v0.6.1 h1:T0Zw1XM5c1GlpN2HYr2s+m3vr1p2wy+8VN+Z1FKxW38=
cloud.google.com/go/auth v0.6.1/go.mod h1:eFHG7zDzbXHKmjJddFG/rBlcGp6t25SwRUiEQSlO4x4=
cloud.google.com/go/auth/oauth2adapt v0.2.2 h1:+TTV8aXpjeChS9M+aTtN/TjdQnzJvmzKFt//oWu7HX4=
cloud.google.com/go/auth/oauth2adapt v0.2.2/go.mod h1:wcYjgpZI9+Yu7LyYBg4pqSiaRkfEK3GQcpb7C/uyF1Q=
cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o=
cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE=
cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc=
cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg=
cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc=
cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ=
cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc=
cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k=
cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE=
cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk=
cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk=
cloud.google.com/go/iam v1.1.8 h1:r7umDwhj+BQyz0ScZMp4QrGXjSTI3ZINnpgU2nlB/K0=
cloud.google.com/go/iam v1.1.8/go.mod h1:GvE6lyMmfxXauzNq8NbgJbeVQNspG+tcdL/W8QO1+zE=
cloud.google.com/go/longrunning v0.5.7 h1:WLbHekDbjK1fVFD3ibpFFVoyizlLRl73I7YKuAKilhU=
cloud.google.com/go/longrunning v0.5.7/go.mod h1:8GClkudohy1Fxm3owmBGid8W0pSgodEMwEAztp38Xng=
cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I=
cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw=
cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA=
@ -36,6 +48,8 @@ cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0Zeo
cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk=
cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs=
cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0=
cloud.google.com/go/storage v1.43.0 h1:CcxnSohZwizt4LCzQHWvBf1/kvtHUn7gk9QERXPyXFs=
cloud.google.com/go/storage v1.43.0/go.mod h1:ajvxEa7WmZS1PxvKRq4bq0tFT3vMd502JwstCcYv0Q0=
dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 h1:/vQbFIOMbk2FiG/kXiLl8BRyzTWDw7gX/Hz7Dd5eDMs=
github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4/go.mod h1:hN7oaIRCjzsZ2dE+yG5k+rsdt3qcwykqK6HVGcKwsw4=
@ -44,6 +58,18 @@ github.com/99designs/keyring v1.2.1/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwR
github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
github.com/AthenZ/athenz v1.10.39 h1:mtwHTF/v62ewY2Z5KWhuZgVXftBej1/Tn80zx4DcawY=
github.com/AthenZ/athenz v1.10.39/go.mod h1:3Tg8HLsiQZp81BJY58JBeU2BR6B/H4/0MQGfCwhHNEA=
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 h1:E+OJmp2tPvt1W+amx48v1eqbjDYsgN+RzP4q16yV5eM=
github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1/go.mod h1:a6xsAQUZg+VsS3TJ05SRp524Hs4pZ/AeFSr5ENf0Yjo=
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 h1:U2rTu3Ef+7w9FHKIAXM6ZyqF3UOWJZ12zIm8zECAFfg=
github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0/go.mod h1:9kIvujWAA58nmPmWB1m23fyWic1kYZMxD9CxaWn4Qpg=
github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 h1:jBQA3cKT4L2rWMpgE7Yt3Hwh2aUj8KXjIGLxjHeYNNo=
github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0/go.mod h1:4OG6tQ9EOP/MT0NMjDlRzWoVFxfu9rN9B2X+tlSVktg=
github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.2.0 h1:Ma67P/GGprNwsslzEH6+Kb8nybI8jpDTm4Wmzu2ReK8=
github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.2.0/go.mod h1:c+Lifp3EDEamAkPVzMooRNOK6CZjNSdEnf1A7jsI9u4=
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0 h1:nVocQV40OQne5613EeLayJiRAJuKlBGy+m22qWG+WRg=
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0/go.mod h1:7QJP7dr2wznCMeqIrhMgWGf7XpAQnVrJqDm9nvV3Cu4=
github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU=
github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI=
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
github.com/BurntSushi/toml v1.2.1 h1:9F2/+DoOYIOksmaJFPw1tGFy1eDnIJXg+UHjuD8lTak=
github.com/BurntSushi/toml v1.2.1/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ=
@ -66,6 +92,12 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy
github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho=
github.com/alibabacloud-go/debug v0.0.0-20190504072949-9472017b5c68 h1:NqugFkGxx1TXSh/pBcU00Y6bljgDPaFdh5MUSeJ7e50=
github.com/alibabacloud-go/debug v0.0.0-20190504072949-9472017b5c68/go.mod h1:6pb/Qy8c+lqua8cFpEy7g39NRRqOWc3rOwAy8m5Y2BY=
github.com/alibabacloud-go/tea v1.1.8 h1:vFF0707fqjGiQTxrtMnIXRjOCvQXf49CuDVRtTopmwU=
github.com/alibabacloud-go/tea v1.1.8/go.mod h1:/tmnEaQMyb4Ky1/5D+SE1BAsa5zj/KeGOFfwYm3N/p4=
github.com/aliyun/credentials-go v1.2.7 h1:gLtFylxLZ1TWi1pStIt1O6a53GFU1zkNwjtJir2B4ow=
github.com/aliyun/credentials-go v1.2.7/go.mod h1:/KowD1cfGSLrLsH28Jr8W+xwoId0ywIy5lNzDz6O1vw=
github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q=
github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY=
github.com/ardielle/ardielle-go v1.5.2 h1:TilHTpHIQJ27R1Tl/iITBzMwiUGSlVfiVhwDNGM3Zj4=
@ -161,8 +193,9 @@ github.com/dimfeld/httptreemux v5.0.1+incompatible h1:Qj3gVcDNoOthBAqftuD596rm4w
github.com/dimfeld/httptreemux v5.0.1+incompatible/go.mod h1:rbUlSV+CCpv/SuqUTP/8Bk2O3LyUV436/yaRGkhP6Z0=
github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw=
github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo=
github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY=
github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto=
github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA2EjTY=
github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU=
github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM=
@ -188,6 +221,8 @@ github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5Kwzbycv
github.com/fatih/color v1.10.0 h1:s36xzo75JdqLaaWoiEHk767eHiwo0598uUxyfiPkDsg=
github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM=
github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M=
github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg=
github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U=
github.com/form3tech-oss/jwt-go v3.2.3+incompatible h1:7ZaBxOI7TMoYBfyA3cQHErNNyAWIKUMIwqxEtgHOs5c=
github.com/form3tech-oss/jwt-go v3.2.3+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k=
github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw=
@ -215,6 +250,8 @@ github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm
github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
github.com/go-ini/ini v1.67.0 h1:z6ZrTEZqSWOTyH2FlglNbNgARyHG8oLW9gMELqKr06A=
github.com/go-ini/ini v1.67.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8=
github.com/go-kit/kit v0.1.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY=
github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
@ -233,6 +270,8 @@ github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/me
github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo=
github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw=
github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM=
github.com/goccy/go-json v0.10.3 h1:KZ5WoDbxAIgm2HNbYckL0se1fHD6rz5j4ywS6ebzDqA=
github.com/goccy/go-json v0.10.3/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M=
github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 h1:ZpnhV/YsD2/4cESfV5+Hoeu/iUR3ruzNvZ+yQfO03a0=
github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4=
github.com/godbus/dbus/v5 v5.0.4 h1:9349emZab16e7zQvpmsbtjc18ykshndd8y2PG3sgJbA=
@ -249,6 +288,8 @@ github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY
github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I=
github.com/golang-jwt/jwt v3.2.2+incompatible h1:IfV12K8xAKAnZqdXVzCZ+TOjboZ2keLg81eXfW3O+oY=
github.com/golang-jwt/jwt v3.2.2+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I=
github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk=
github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
github.com/golang/glog v1.2.1 h1:OptwRhECazUx5ix5TTWC3EZhsZEHWcYWY4FQHTIubm4=
github.com/golang/glog v1.2.1/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w=
@ -256,6 +297,7 @@ github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4er
github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE=
github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
@ -314,9 +356,12 @@ github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0=
github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no=
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0=
github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0=
github.com/google/martian/v3 v3.3.3 h1:DIhPTQrbPkgs2yJYdXU/eNACCG5DVQjySNRNlflZ9Fc=
github.com/google/martian/v3 v3.3.3/go.mod h1:iEPrYcgCF7jA9OtScMFQyAlZZ4YXTKEtJ1E6RWzmBA0=
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM=
@ -330,13 +375,21 @@ github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLe
github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
github.com/google/pprof v0.0.0-20211008130755-947d60d73cc0/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg=
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o=
github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw=
github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0=
github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs=
github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0=
github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk=
github.com/googleapis/gax-go/v2 v2.12.5 h1:8gw9KZK8TiVKB6q3zHY3SBzLnrGp6HQjyfYBYGmXdxA=
github.com/googleapis/gax-go/v2 v2.12.5/go.mod h1:BUDKcWo+RaKq5SC9vVYL0wLADa3VcfswbOMMRmB9H3E=
github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
github.com/gopherjs/gopherjs v0.0.0-20200217142428-fce0ec30dd00 h1:l5lAOZEym3oK3SQ2HBHWsJUfbNBiTXJDeW2QDxw9AQ0=
github.com/gopherjs/gopherjs v0.0.0-20200217142428-fce0ec30dd00/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So=
github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc=
@ -418,6 +471,7 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr
github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo=
github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk=
github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo=
github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
github.com/juju/qthttptest v0.1.1/go.mod h1:aTlAv8TYaflIiTDIQYzxnl1QdPjAg8Q8qJMErpKy6A4=
github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w=
@ -434,9 +488,12 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
github.com/klauspost/compress v1.17.7 h1:ehO88t2UGzQK66LMdE8tibEd1ErmzZjNEqWkjLAKQQg=
github.com/klauspost/compress v1.17.7/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw=
github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA=
github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw=
github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
github.com/klauspost/cpuid/v2 v2.0.1/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg=
github.com/klauspost/cpuid/v2 v2.2.8 h1:+StwCXwm9PdpiEkPyzBXIy+M9KUb4ODm0Zarf1kS5BM=
github.com/klauspost/cpuid/v2 v2.2.8/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws=
github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
@ -453,6 +510,8 @@ github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 h1:vN4d3jSss3ExzU
github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06/go.mod h1:++9BgZujZd4v0ZTZCb5iPsaomXdZWyxotIAh1IiDm44=
github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b h1:xYEM2oBUhBEhQjrV+KJ9lEWDWYZoNVZUaBF++Wyljq4=
github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b/go.mod h1:V0HF/ZBlN86HqewcDC/cVxMmYDiRukWjSrgKLUAn9Js=
github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc=
github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw=
github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y=
github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k=
github.com/linkedin/goavro v2.1.0+incompatible/go.mod h1:bBCwI2eGYpUI/4820s67MElg9tdeLbINjLjiM2xZFYM=
@ -498,6 +557,10 @@ github.com/milvus-io/pulsar-client-go v0.12.1 h1:O2JZp1tsYiO7C0MQ4hrUY/aJXnn2Gry
github.com/milvus-io/pulsar-client-go v0.12.1/go.mod h1:dkutuH4oS2pXiGm+Ti7fQZ4MRjrMPZ8IJeEGAWMeckk=
github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g=
github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY=
github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34=
github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM=
github.com/minio/minio-go/v7 v7.0.73 h1:qr2vi96Qm7kZ4v7LLebjte+MQh621fFWnv93p12htEo=
github.com/minio/minio-go/v7 v7.0.73/go.mod h1:qydcVzV8Hqtj1VtEocfxbmVFa2siu6HGa+LDEPogjD8=
github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc=
github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
@ -571,6 +634,8 @@ github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a h1:LzIZsQpXQlj8yF7
github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI=
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 h1:URLoJ61DmmY++Sa/yyPEQHG2s/ZBeV1FbIswHEMrdoY=
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ=
github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU=
github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
@ -626,8 +691,10 @@ github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTE
github.com/rogpeppe/go-internal v1.8.0/go.mod h1:WmiCO8CzOY8rg0OYDC4/i/2WRWAB6poM+XZ2dLUbcbE=
github.com/rogpeppe/go-internal v1.8.1/go.mod h1:JeRgkft04UBgHMgCIwADu4Pn6Mtm5d4nPKWu0nJ5d+o=
github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs=
github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ=
github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog=
github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8=
github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4=
github.com/rs/xid v1.5.0 h1:mKX4bl4iPYJtEIxp6CYiUuLQ/8DYMoz0PUdtGgMFRVc=
github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg=
github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g=
github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
@ -651,6 +718,9 @@ github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic
github.com/sirupsen/logrus v1.9.0 h1:trlNQbNUG3OdDrDil03MCb1H2o9nJ1x4/5LYw7byDE0=
github.com/sirupsen/logrus v1.9.0/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ=
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
github.com/smartystreets/assertions v1.1.0 h1:MkTeG1DMwsrdH7QtLXy5W+fUxWq+vmb6cLmyJ7aRtF0=
github.com/smartystreets/assertions v1.1.0/go.mod h1:tcbTF8ujkAEcZ8TElKY+i30BzYlVhC/LOxJk7iOWnoo=
github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s=
github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js=
@ -694,6 +764,8 @@ github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o
github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg=
github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY=
github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw=
github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.865 h1:LcUqBlKC4j15LhT303yQDX/XxyHG4haEQqbHgZZA4SY=
github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.865/go.mod h1:r5r4xbfxSaeR04b166HGsBa/R4U3SueirEUpXGuw+Q0=
github.com/thoas/go-funk v0.9.1 h1:O549iLZqPpTUQ10ykd26sZhzD+rmR5pWhuElrhbC20M=
github.com/thoas/go-funk v0.9.1/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q=
github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4=
@ -777,9 +849,13 @@ go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk=
go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E=
go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0=
go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo=
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.25.0/go.mod h1:E5NNboN0UqSAki0Atn9kVwaN7I+l25gGxDqBueo/74E=
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 h1:4Pp6oUg3+e/6M4C0A/3kJ2VYa++dsWVTtGgLVj5xtHg=
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0/go.mod h1:Mjt1i1INqiaoZOMGR1RIUJN+i3ChKoFRqzrRQhlkbs0=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 h1:jq9TW8u3so/bN+JPT166wjOI6/vQPF6Xe7nMNIltagk=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0/go.mod h1:p8pYQP+m5XfbZm9fxtSKAbM6oIllS7s2AfxrChvc7iw=
go.opentelemetry.io/otel v1.0.1/go.mod h1:OPEOD4jIT2SlZPMmwT6FqZz2C0ZNdQqiWcoK6M0SNFU=
go.opentelemetry.io/otel v1.28.0 h1:/SqNcYk+idO0CxKEUOtKQClMK/MimZihKYMruSMViUo=
go.opentelemetry.io/otel v1.28.0/go.mod h1:q68ijF8Fc8CnMHKyzqL6akLO46ePnjkgfIMIjUIX9z4=
@ -946,8 +1022,8 @@ golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ
golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A=
golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A=
golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A=
golang.org/x/oauth2 v0.20.0 h1:4mQdhULixXKP1rwYBW0vAijoXnkTG0BLCDRzfe1idMo=
golang.org/x/oauth2 v0.20.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI=
golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs=
golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI=
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
@ -1033,6 +1109,8 @@ golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20220209214540-3681064d5158/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA=
golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA=
@ -1143,6 +1221,8 @@ google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjR
google.golang.org/api v0.41.0/go.mod h1:RkxM5lITDfTzmyKFPt+wGrCJbVfniCr2ool8kTBzRTU=
google.golang.org/api v0.43.0/go.mod h1:nQsDGjRXMo4lvh5hP0TKqF244gqhGcr/YSIykhUk/94=
google.golang.org/api v0.44.0/go.mod h1:EBOGZqzyhtvMDoxwS97ctnh0zUmYY6CxqXsc1AvkYD8=
google.golang.org/api v0.187.0 h1:Mxs7VATVC2v7CY+7Xwm4ndkX71hpElcvx0D1Ji/p1eo=
google.golang.org/api v0.187.0/go.mod h1:KIHlTc4x7N7gKKuVsdmfBXN13yEEWXWFURWY6SBp2gk=
google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
@ -1196,10 +1276,10 @@ google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaE
google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0=
google.golang.org/genproto v0.0.0-20210624195500-8bfb893ecb84/go.mod h1:SzzZ/N+nwJDaO1kznhnlzqS8ocJICar6hYhVyhi++24=
google.golang.org/genproto v0.0.0-20220503193339-ba3ae3f07e29/go.mod h1:RAyBrSAP7Fh3Nc84ghnVLDPuV51xc9agzmm4Ph6i0Q4=
google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 h1:wpZ8pe2x1Q3f2KyT5f8oP/fa9rHAKgFPr/HZdNuS+PQ=
google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:J7XzRzVy1+IPwWHZUzoD0IccYZIrXILAQpc+Qy9CMhY=
google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157 h1:7whR9kGa5LUwFtpLm2ArCEejtnxlGeLbAyjFY8sGNFw=
google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU=
google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d h1:PksQg4dV6Sem3/HkBX+Ltq8T0ke0PKIRBNBatoDTVls=
google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d/go.mod h1:s7iA721uChleev562UJO2OYB0PPT9CMFjV+Ce7VJH5M=
google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4 h1:MuYw1wJzT+ZkybKfaOXKp5hJiZDn2iHaXRw0mRYdHSc=
google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4/go.mod h1:px9SlOOZBg1wM1zdnr8jEL4CNGUBZ+ZKYtNPApNQc4c=
google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf h1:liao9UHurZLtiEwBgT9LMOnKYsHze6eA6w1KQCMVN2Q=
google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY=
google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw=
@ -1252,6 +1332,7 @@ gopkg.in/avro.v0 v0.0.0-20171217001914-a730b5802183/go.mod h1:FvqrFXt+jCsyQibeRv
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
@ -1265,6 +1346,8 @@ gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc=
gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
gopkg.in/ini.v1 v1.56.0 h1:DPMeDvGTM54DXbPkVIZsp19fp/I2K7zwA/itHYHKo8Y=
gopkg.in/ini.v1 v1.56.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA=
gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA=
gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8=

View File

@ -9,7 +9,7 @@ import (
"github.com/minio/minio-go/v7/pkg/credentials"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/storage/aliyun/mocks"
"github.com/milvus-io/milvus/pkg/v2/objectstorage/aliyun/mocks"
)
func TestNewMinioClient(t *testing.T) {

View File

@ -2,7 +2,7 @@
package mocks
import mock "github.com/stretchr/testify/mock"
import "github.com/stretchr/testify/mock"
// Credential is an autogenerated mock type for the Credential type
type Credential struct {

View File

@ -0,0 +1,118 @@
package objectstorage
// Config for setting params used by chunk manager client.
type Config struct {
Address string
BucketName string
AccessKeyID string
SecretAccessKeyID string
UseSSL bool
SslCACert string
CreateBucket bool
RootPath string
UseIAM bool
CloudProvider string
IAMEndpoint string
UseVirtualHost bool
Region string
RequestTimeoutMs int64
GcpCredentialJSON string
GcpNativeWithoutAuth bool // used for Unit Testing
}
func NewDefaultConfig() *Config {
return &Config{}
}
// Option is used to Config the retry function.
type Option func(*Config)
func Address(addr string) Option {
return func(c *Config) {
c.Address = addr
}
}
func BucketName(bucketName string) Option {
return func(c *Config) {
c.BucketName = bucketName
}
}
func AccessKeyID(accessKeyID string) Option {
return func(c *Config) {
c.AccessKeyID = accessKeyID
}
}
func SecretAccessKeyID(secretAccessKeyID string) Option {
return func(c *Config) {
c.SecretAccessKeyID = secretAccessKeyID
}
}
func UseSSL(useSSL bool) Option {
return func(c *Config) {
c.UseSSL = useSSL
}
}
func SslCACert(sslCACert string) Option {
return func(c *Config) {
c.SslCACert = sslCACert
}
}
func CreateBucket(createBucket bool) Option {
return func(c *Config) {
c.CreateBucket = createBucket
}
}
func RootPath(rootPath string) Option {
return func(c *Config) {
c.RootPath = rootPath
}
}
func UseIAM(useIAM bool) Option {
return func(c *Config) {
c.UseIAM = useIAM
}
}
func CloudProvider(cloudProvider string) Option {
return func(c *Config) {
c.CloudProvider = cloudProvider
}
}
func IAMEndpoint(iamEndpoint string) Option {
return func(c *Config) {
c.IAMEndpoint = iamEndpoint
}
}
func UseVirtualHost(useVirtualHost bool) Option {
return func(c *Config) {
c.UseVirtualHost = useVirtualHost
}
}
func Region(region string) Option {
return func(c *Config) {
c.Region = region
}
}
func RequestTimeout(requestTimeoutMs int64) Option {
return func(c *Config) {
c.RequestTimeoutMs = requestTimeoutMs
}
}
func GcpCredentialJSON(gcpCredentialJSON string) Option {
return func(c *Config) {
c.GcpCredentialJSON = gcpCredentialJSON
}
}

281
pkg/objectstorage/util.go Normal file
View File

@ -0,0 +1,281 @@
package objectstorage
import (
"context"
"encoding/json"
"fmt"
"os"
"strings"
"cloud.google.com/go/storage"
"github.com/Azure/azure-sdk-for-go/sdk/azcore"
"github.com/Azure/azure-sdk-for-go/sdk/azidentity"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/bloberror"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/container"
"github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/service"
"github.com/cockroachdb/errors"
"github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials"
"go.uber.org/zap"
"golang.org/x/oauth2/google"
"google.golang.org/api/option"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/objectstorage/aliyun"
"github.com/milvus-io/milvus/pkg/v2/objectstorage/gcp"
"github.com/milvus-io/milvus/pkg/v2/objectstorage/tencent"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/retry"
)
const (
CloudProviderGCP = "gcp"
CloudProviderGCPNative = "gcpnative"
CloudProviderAWS = "aws"
CloudProviderAliyun = "aliyun"
CloudProviderAzure = "azure"
CloudProviderTencent = "tencent"
)
var CheckBucketRetryAttempts uint = 20
func NewMinioClient(ctx context.Context, c *Config) (*minio.Client, error) {
var creds *credentials.Credentials
newMinioFn := minio.New
bucketLookupType := minio.BucketLookupAuto
if c.UseVirtualHost {
bucketLookupType = minio.BucketLookupDNS
}
matchedDefault := false
switch c.CloudProvider {
case CloudProviderAliyun:
// auto doesn't work for aliyun, so we set to dns deliberately
bucketLookupType = minio.BucketLookupDNS
if c.UseIAM {
newMinioFn = aliyun.NewMinioClient
} else {
creds = credentials.NewStaticV4(c.AccessKeyID, c.SecretAccessKeyID, "")
}
case CloudProviderGCP:
newMinioFn = gcp.NewMinioClient
if !c.UseIAM {
creds = credentials.NewStaticV2(c.AccessKeyID, c.SecretAccessKeyID, "")
}
case CloudProviderTencent:
bucketLookupType = minio.BucketLookupDNS
newMinioFn = tencent.NewMinioClient
if !c.UseIAM {
creds = credentials.NewStaticV4(c.AccessKeyID, c.SecretAccessKeyID, "")
}
default: // aws, minio
matchedDefault = true
}
// Compatibility logic. If the cloud provider is not specified in the request,
// it shall be inferred based on the service address.
if matchedDefault {
matchedDefault = false
switch {
case strings.Contains(c.Address, gcp.GcsDefaultAddress):
newMinioFn = gcp.NewMinioClient
if !c.UseIAM {
creds = credentials.NewStaticV2(c.AccessKeyID, c.SecretAccessKeyID, "")
}
case strings.Contains(c.Address, aliyun.OSSAddressFeatureString):
// auto doesn't work for aliyun, so we set to dns deliberately
bucketLookupType = minio.BucketLookupDNS
if c.UseIAM {
newMinioFn = aliyun.NewMinioClient
} else {
creds = credentials.NewStaticV4(c.AccessKeyID, c.SecretAccessKeyID, "")
}
default:
matchedDefault = true
}
}
if matchedDefault {
// aws, minio
if c.UseIAM {
creds = credentials.NewIAM("")
} else {
creds = credentials.NewStaticV4(c.AccessKeyID, c.SecretAccessKeyID, "")
}
}
// We must set the cert path by os environment variable "SSL_CERT_FILE",
// because the minio.DefaultTransport() need this path to read the file content,
// we shouldn't read this file by ourself.
if c.UseSSL && len(c.SslCACert) > 0 {
err := os.Setenv("SSL_CERT_FILE", c.SslCACert)
if err != nil {
return nil, err
}
}
minioOpts := &minio.Options{
BucketLookup: bucketLookupType,
Creds: creds,
Secure: c.UseSSL,
Region: c.Region,
}
minIOClient, err := newMinioFn(c.Address, minioOpts)
// options nil or invalid formatted endpoint, don't need to retry
if err != nil {
return nil, err
}
var bucketExists bool
// check valid in first query
checkBucketFn := func() error {
bucketExists, err = minIOClient.BucketExists(ctx, c.BucketName)
if err != nil {
log.Warn("failed to check blob bucket exist", zap.String("bucket", c.BucketName), zap.Error(err))
return err
}
if !bucketExists {
if c.CreateBucket {
log.Info("blob bucket not exist, create bucket.", zap.String("bucket name", c.BucketName))
err := minIOClient.MakeBucket(ctx, c.BucketName, minio.MakeBucketOptions{})
if err != nil {
log.Warn("failed to create blob bucket", zap.String("bucket", c.BucketName), zap.Error(err))
return err
}
} else {
return fmt.Errorf("bucket %s not Existed", c.BucketName)
}
}
return nil
}
err = retry.Do(ctx, checkBucketFn, retry.Attempts(CheckBucketRetryAttempts))
if err != nil {
return nil, err
}
return minIOClient, nil
}
func NewAzureObjectStorageClient(ctx context.Context, c *Config) (*service.Client, error) {
var client *service.Client
var err error
if c.UseIAM {
cred, credErr := azidentity.NewWorkloadIdentityCredential(&azidentity.WorkloadIdentityCredentialOptions{
ClientID: os.Getenv("AZURE_CLIENT_ID"),
TenantID: os.Getenv("AZURE_TENANT_ID"),
TokenFilePath: os.Getenv("AZURE_FEDERATED_TOKEN_FILE"),
})
if credErr != nil {
return nil, credErr
}
client, err = service.NewClient("https://"+c.AccessKeyID+".blob."+c.Address+"/", cred, &service.ClientOptions{})
} else {
connectionString := os.Getenv("AZURE_STORAGE_CONNECTION_STRING")
if connectionString == "" {
connectionString = "DefaultEndpointsProtocol=https;AccountName=" + c.AccessKeyID +
";AccountKey=" + c.SecretAccessKeyID + ";EndpointSuffix=" + c.Address
}
client, err = service.NewClientFromConnectionString(connectionString, &service.ClientOptions{})
}
if err != nil {
return nil, err
}
if c.BucketName == "" {
return nil, merr.WrapErrParameterInvalidMsg("invalid empty bucket name")
}
// check valid in first query
checkBucketFn := func() error {
_, err := client.NewContainerClient(c.BucketName).GetProperties(ctx, &container.GetPropertiesOptions{})
if err != nil {
switch err := err.(type) {
case *azcore.ResponseError:
if c.CreateBucket && err.ErrorCode == string(bloberror.ContainerNotFound) {
_, createErr := client.NewContainerClient(c.BucketName).Create(ctx, &azblob.CreateContainerOptions{})
if createErr != nil {
return createErr
}
return nil
}
}
}
return err
}
err = retry.Do(ctx, checkBucketFn, retry.Attempts(CheckBucketRetryAttempts))
if err != nil {
return nil, err
}
return client, nil
}
func NewGcpObjectStorageClient(ctx context.Context, c *Config) (*storage.Client, error) {
var err error
var opts []option.ClientOption
var projectId string
if c.Address != "" {
completeAddress := "http://"
if c.UseSSL {
completeAddress = "https://"
}
completeAddress = completeAddress + c.Address + "/storage/v1/"
opts = append(opts, option.WithEndpoint(completeAddress))
}
if c.GcpNativeWithoutAuth {
opts = append(opts, option.WithoutAuthentication())
} else {
creds, err := google.CredentialsFromJSON(ctx, []byte(c.GcpCredentialJSON), storage.ScopeReadWrite)
if err != nil {
return nil, err
}
projectId, err = getProjectId(c.GcpCredentialJSON)
if err != nil {
return nil, err
}
opts = append(opts, option.WithCredentials(creds))
}
client, err := storage.NewClient(ctx, opts...)
if err != nil {
return nil, err
}
if c.BucketName == "" {
return nil, merr.WrapErrParameterInvalidMsg("invalid empty bucket name")
}
// Check bucket validity
checkBucketFn := func() error {
bucket := client.Bucket(c.BucketName)
_, err = bucket.Attrs(ctx)
if errors.Is(err, storage.ErrBucketNotExist) && c.CreateBucket {
log.Info("gcs bucket does not exist, create bucket.", zap.String("bucket name", c.BucketName))
err = client.Bucket(c.BucketName).Create(ctx, projectId, nil)
if err != nil {
return err
}
return nil
}
return err
}
err = retry.Do(ctx, checkBucketFn, retry.Attempts(CheckBucketRetryAttempts))
if err != nil {
return nil, err
}
return client, nil
}
func getProjectId(gcpCredentialJSON string) (string, error) {
if gcpCredentialJSON == "" {
return "", errors.New("the JSON string is empty")
}
var data map[string]interface{}
if err := json.Unmarshal([]byte(gcpCredentialJSON), &data); err != nil {
return "", errors.New("failed to parse Google Cloud credentials as JSON")
}
propertyValue, ok := data["project_id"]
projectId := fmt.Sprintf("%v", propertyValue)
if !ok {
return "", errors.New("projectId doesn't exist")
}
return projectId, nil
}