aboutsummaryrefslogtreecommitdiff
path: root/test
diff options
context:
space:
mode:
Diffstat (limited to 'test')
-rw-r--r--test/erasure_coding/ec_integration_test.go438
-rw-r--r--test/s3/sse/s3_sse_integration_test.go72
-rw-r--r--test/s3/tagging/Makefile321
-rw-r--r--test/s3/tagging/README.md53
-rw-r--r--test/s3/tagging/s3_tagging_test.go446
-rw-r--r--test/sftp/Makefile41
-rw-r--r--test/sftp/README.md92
-rw-r--r--test/sftp/basic_test.go652
-rw-r--r--test/sftp/framework.go423
-rw-r--r--test/sftp/go.mod17
-rw-r--r--test/sftp/go.sum64
-rw-r--r--test/sftp/testdata/userstore.json37
12 files changed, 2655 insertions, 1 deletions
diff --git a/test/erasure_coding/ec_integration_test.go b/test/erasure_coding/ec_integration_test.go
index 87b9b40ba..67f8eed04 100644
--- a/test/erasure_coding/ec_integration_test.go
+++ b/test/erasure_coding/ec_integration_test.go
@@ -5,9 +5,11 @@ import (
"context"
"fmt"
"io"
+ "math"
"os"
"os/exec"
"path/filepath"
+ "strings"
"testing"
"time"
@@ -139,6 +141,15 @@ func TestECEncodingVolumeLocationTimingBug(t *testing.T) {
t.Logf("EC encoding completed successfully")
}
+ // Add detailed logging for EC encoding command
+ t.Logf("Debug: Executing EC encoding command for volume %d", volumeId)
+ t.Logf("Debug: Command arguments: %v", args)
+ if err != nil {
+ t.Logf("Debug: EC encoding command failed with error: %v", err)
+ } else {
+ t.Logf("Debug: EC encoding command completed successfully")
+ }
+
// The key test: check if the fix prevents the timing issue
if contains(outputStr, "Collecting volume locations") && contains(outputStr, "before EC encoding") {
t.Logf("FIX DETECTED: Volume locations collected BEFORE EC encoding (timing bug prevented)")
@@ -526,7 +537,8 @@ func uploadTestData(data []byte, masterAddress string) (needle.VolumeId, error)
func getVolumeLocations(commandEnv *shell.CommandEnv, volumeId needle.VolumeId) ([]string, error) {
// Retry mechanism to handle timing issues with volume registration
- for i := 0; i < 10; i++ {
+ // Increase retry attempts for volume location retrieval
+ for i := 0; i < 20; i++ { // Increased from 10 to 20 retries
locations, ok := commandEnv.MasterClient.GetLocationsClone(uint32(volumeId))
if ok {
var result []string
@@ -646,3 +658,427 @@ func TestECEncodingRegressionPrevention(t *testing.T) {
t.Log("Timing pattern regression test passed")
})
}
+
+// TestDiskAwareECRebalancing tests EC shard placement across multiple disks per server
+// This verifies the disk-aware EC rebalancing feature works correctly
+func TestDiskAwareECRebalancing(t *testing.T) {
+ if testing.Short() {
+ t.Skip("Skipping disk-aware integration test in short mode")
+ }
+
+ testDir, err := os.MkdirTemp("", "seaweedfs_disk_aware_ec_test_")
+ require.NoError(t, err)
+ defer os.RemoveAll(testDir)
+
+ ctx, cancel := context.WithTimeout(context.Background(), 180*time.Second)
+ defer cancel()
+
+ // Start cluster with MULTIPLE DISKS per volume server
+ cluster, err := startMultiDiskCluster(ctx, testDir)
+ require.NoError(t, err)
+ defer cluster.Stop()
+
+ // Wait for servers to be ready
+ require.NoError(t, waitForServer("127.0.0.1:9334", 30*time.Second))
+ for i := 0; i < 3; i++ {
+ require.NoError(t, waitForServer(fmt.Sprintf("127.0.0.1:809%d", i), 30*time.Second))
+ }
+
+ // Wait longer for volume servers to register with master and create volumes
+ t.Log("Waiting for volume servers to register with master...")
+ time.Sleep(10 * time.Second)
+
+ // Create command environment
+ options := &shell.ShellOptions{
+ Masters: stringPtr("127.0.0.1:9334"),
+ GrpcDialOption: grpc.WithInsecure(),
+ FilerGroup: stringPtr("default"),
+ }
+ commandEnv := shell.NewCommandEnv(options)
+
+ // Connect to master with longer timeout
+ ctx2, cancel2 := context.WithTimeout(context.Background(), 60*time.Second)
+ defer cancel2()
+ go commandEnv.MasterClient.KeepConnectedToMaster(ctx2)
+ commandEnv.MasterClient.WaitUntilConnected(ctx2)
+
+ // Wait for master client to fully sync
+ time.Sleep(5 * time.Second)
+
+ // Upload test data to create a volume - retry if volumes not ready
+ var volumeId needle.VolumeId
+ testData := []byte("Disk-aware EC rebalancing test data - this needs to be large enough to create a volume")
+ for retry := 0; retry < 5; retry++ {
+ volumeId, err = uploadTestDataToMaster(testData, "127.0.0.1:9334")
+ if err == nil {
+ break
+ }
+ t.Logf("Upload attempt %d failed: %v, retrying...", retry+1, err)
+ time.Sleep(3 * time.Second)
+ }
+ require.NoError(t, err, "Failed to upload test data after retries")
+ t.Logf("Created volume %d for disk-aware EC test", volumeId)
+
+ // Wait for volume to be registered
+ time.Sleep(3 * time.Second)
+
+ t.Run("verify_multi_disk_setup", func(t *testing.T) {
+ // Verify that each server has multiple disk directories
+ for server := 0; server < 3; server++ {
+ diskCount := 0
+ for disk := 0; disk < 4; disk++ {
+ diskDir := filepath.Join(testDir, fmt.Sprintf("server%d_disk%d", server, disk))
+ if _, err := os.Stat(diskDir); err == nil {
+ diskCount++
+ }
+ }
+ assert.Equal(t, 4, diskCount, "Server %d should have 4 disk directories", server)
+ t.Logf("Server %d has %d disk directories", server, diskCount)
+ }
+ })
+
+ t.Run("ec_encode_with_disk_awareness", func(t *testing.T) {
+ // Get lock first
+ lockCmd := shell.Commands[findCommandIndex("lock")]
+ var lockOutput bytes.Buffer
+ err := lockCmd.Do([]string{}, commandEnv, &lockOutput)
+ if err != nil {
+ t.Logf("Lock command failed: %v", err)
+ }
+
+ // Execute EC encoding
+ var output bytes.Buffer
+ ecEncodeCmd := shell.Commands[findCommandIndex("ec.encode")]
+ args := []string{"-volumeId", fmt.Sprintf("%d", volumeId), "-collection", "test", "-force"}
+
+ // Capture output
+ oldStdout := os.Stdout
+ oldStderr := os.Stderr
+ r, w, _ := os.Pipe()
+ os.Stdout = w
+ os.Stderr = w
+
+ err = ecEncodeCmd.Do(args, commandEnv, &output)
+
+ w.Close()
+ os.Stdout = oldStdout
+ os.Stderr = oldStderr
+
+ capturedOutput, _ := io.ReadAll(r)
+ outputStr := string(capturedOutput) + output.String()
+
+ t.Logf("EC encode output:\n%s", outputStr)
+
+ if err != nil {
+ t.Logf("EC encoding completed with error: %v", err)
+ } else {
+ t.Logf("EC encoding completed successfully")
+ }
+ })
+
+ t.Run("verify_disk_level_shard_distribution", func(t *testing.T) {
+ // Wait for shards to be distributed
+ time.Sleep(2 * time.Second)
+
+ // Count shards on each disk of each server
+ diskDistribution := countShardsPerDisk(testDir, uint32(volumeId))
+
+ totalShards := 0
+ disksWithShards := 0
+ maxShardsOnSingleDisk := 0
+
+ t.Logf("Disk-level shard distribution for volume %d:", volumeId)
+ for server, disks := range diskDistribution {
+ for diskId, shardCount := range disks {
+ if shardCount > 0 {
+ t.Logf(" %s disk %d: %d shards", server, diskId, shardCount)
+ totalShards += shardCount
+ disksWithShards++
+ if shardCount > maxShardsOnSingleDisk {
+ maxShardsOnSingleDisk = shardCount
+ }
+ }
+ }
+ }
+
+ t.Logf("Summary: %d total shards across %d disks (max %d on single disk)",
+ totalShards, disksWithShards, maxShardsOnSingleDisk)
+
+ // EC creates 14 shards (10 data + 4 parity), plus .ecx and .ecj files
+ // We should see shards distributed across multiple disks
+ if disksWithShards > 1 {
+ t.Logf("PASS: Shards distributed across %d disks", disksWithShards)
+ } else {
+ t.Logf("INFO: Shards on %d disk(s) - may be expected if volume was on single disk", disksWithShards)
+ }
+ })
+
+ t.Run("test_ec_balance_disk_awareness", func(t *testing.T) {
+ // Calculate initial disk balance variance
+ initialDistribution := countShardsPerDisk(testDir, uint32(volumeId))
+ initialVariance := calculateDiskShardVariance(initialDistribution)
+ t.Logf("Initial disk shard variance: %.2f", initialVariance)
+
+ // Run ec.balance command
+ var output bytes.Buffer
+ ecBalanceCmd := shell.Commands[findCommandIndex("ec.balance")]
+
+ oldStdout := os.Stdout
+ oldStderr := os.Stderr
+ r, w, _ := os.Pipe()
+ os.Stdout = w
+ os.Stderr = w
+
+ err := ecBalanceCmd.Do([]string{"-force"}, commandEnv, &output)
+
+ w.Close()
+ os.Stdout = oldStdout
+ os.Stderr = oldStderr
+
+ capturedOutput, _ := io.ReadAll(r)
+ outputStr := string(capturedOutput) + output.String()
+
+ if err != nil {
+ t.Logf("ec.balance error: %v", err)
+ }
+ t.Logf("ec.balance output:\n%s", outputStr)
+
+ // Wait for balance to complete
+ time.Sleep(2 * time.Second)
+
+ // Calculate final disk balance variance
+ finalDistribution := countShardsPerDisk(testDir, uint32(volumeId))
+ finalVariance := calculateDiskShardVariance(finalDistribution)
+ t.Logf("Final disk shard variance: %.2f", finalVariance)
+
+ t.Logf("Variance change: %.2f -> %.2f", initialVariance, finalVariance)
+ })
+
+ t.Run("verify_no_disk_overload", func(t *testing.T) {
+ // Verify that no single disk has too many shards of the same volume
+ diskDistribution := countShardsPerDisk(testDir, uint32(volumeId))
+
+ for server, disks := range diskDistribution {
+ for diskId, shardCount := range disks {
+ // With 14 EC shards and 12 disks (3 servers x 4 disks), ideally ~1-2 shards per disk
+ // Allow up to 4 shards per disk as a reasonable threshold
+ if shardCount > 4 {
+ t.Logf("WARNING: %s disk %d has %d shards (may indicate imbalance)",
+ server, diskId, shardCount)
+ }
+ }
+ }
+ })
+}
+
+// MultiDiskCluster represents a test cluster with multiple disks per volume server
+type MultiDiskCluster struct {
+ masterCmd *exec.Cmd
+ volumeServers []*exec.Cmd
+ testDir string
+}
+
+func (c *MultiDiskCluster) Stop() {
+ // Stop volume servers first
+ for _, cmd := range c.volumeServers {
+ if cmd != nil && cmd.Process != nil {
+ cmd.Process.Kill()
+ cmd.Wait()
+ }
+ }
+
+ // Stop master server
+ if c.masterCmd != nil && c.masterCmd.Process != nil {
+ c.masterCmd.Process.Kill()
+ c.masterCmd.Wait()
+ }
+}
+
+// startMultiDiskCluster starts a SeaweedFS cluster with multiple disks per volume server
+func startMultiDiskCluster(ctx context.Context, dataDir string) (*MultiDiskCluster, error) {
+ weedBinary := findWeedBinary()
+ if weedBinary == "" {
+ return nil, fmt.Errorf("weed binary not found")
+ }
+
+ cluster := &MultiDiskCluster{testDir: dataDir}
+
+ // Create master directory
+ masterDir := filepath.Join(dataDir, "master")
+ os.MkdirAll(masterDir, 0755)
+
+ // Start master server on a different port to avoid conflict
+ masterCmd := exec.CommandContext(ctx, weedBinary, "master",
+ "-port", "9334",
+ "-mdir", masterDir,
+ "-volumeSizeLimitMB", "10",
+ "-ip", "127.0.0.1",
+ )
+
+ masterLogFile, err := os.Create(filepath.Join(masterDir, "master.log"))
+ if err != nil {
+ return nil, fmt.Errorf("failed to create master log file: %v", err)
+ }
+ masterCmd.Stdout = masterLogFile
+ masterCmd.Stderr = masterLogFile
+
+ if err := masterCmd.Start(); err != nil {
+ return nil, fmt.Errorf("failed to start master server: %v", err)
+ }
+ cluster.masterCmd = masterCmd
+
+ // Wait for master to be ready
+ time.Sleep(2 * time.Second)
+
+ // Start 3 volume servers, each with 4 disks
+ const numServers = 3
+ const disksPerServer = 4
+
+ for i := 0; i < numServers; i++ {
+ // Create 4 disk directories per server
+ var diskDirs []string
+ var maxVolumes []string
+
+ for d := 0; d < disksPerServer; d++ {
+ diskDir := filepath.Join(dataDir, fmt.Sprintf("server%d_disk%d", i, d))
+ if err := os.MkdirAll(diskDir, 0755); err != nil {
+ cluster.Stop()
+ return nil, fmt.Errorf("failed to create disk dir: %v", err)
+ }
+ diskDirs = append(diskDirs, diskDir)
+ maxVolumes = append(maxVolumes, "5")
+ }
+
+ port := fmt.Sprintf("809%d", i)
+ rack := fmt.Sprintf("rack%d", i)
+
+ volumeCmd := exec.CommandContext(ctx, weedBinary, "volume",
+ "-port", port,
+ "-dir", strings.Join(diskDirs, ","),
+ "-max", strings.Join(maxVolumes, ","),
+ "-mserver", "127.0.0.1:9334",
+ "-ip", "127.0.0.1",
+ "-dataCenter", "dc1",
+ "-rack", rack,
+ )
+
+ // Create log file for this volume server
+ logDir := filepath.Join(dataDir, fmt.Sprintf("server%d_logs", i))
+ os.MkdirAll(logDir, 0755)
+ volumeLogFile, err := os.Create(filepath.Join(logDir, "volume.log"))
+ if err != nil {
+ cluster.Stop()
+ return nil, fmt.Errorf("failed to create volume log file: %v", err)
+ }
+ volumeCmd.Stdout = volumeLogFile
+ volumeCmd.Stderr = volumeLogFile
+
+ if err := volumeCmd.Start(); err != nil {
+ cluster.Stop()
+ return nil, fmt.Errorf("failed to start volume server %d: %v", i, err)
+ }
+ cluster.volumeServers = append(cluster.volumeServers, volumeCmd)
+ }
+
+ // Wait for volume servers to register with master
+ // Multi-disk servers may take longer to initialize
+ time.Sleep(8 * time.Second)
+
+ return cluster, nil
+}
+
+// uploadTestDataToMaster uploads test data to a specific master address
+func uploadTestDataToMaster(data []byte, masterAddress string) (needle.VolumeId, error) {
+ assignResult, err := operation.Assign(context.Background(), func(ctx context.Context) pb.ServerAddress {
+ return pb.ServerAddress(masterAddress)
+ }, grpc.WithInsecure(), &operation.VolumeAssignRequest{
+ Count: 1,
+ Collection: "test",
+ Replication: "000",
+ })
+ if err != nil {
+ return 0, err
+ }
+
+ uploader, err := operation.NewUploader()
+ if err != nil {
+ return 0, err
+ }
+
+ uploadResult, err, _ := uploader.Upload(context.Background(), bytes.NewReader(data), &operation.UploadOption{
+ UploadUrl: "http://" + assignResult.Url + "/" + assignResult.Fid,
+ Filename: "testfile.txt",
+ MimeType: "text/plain",
+ })
+ if err != nil {
+ return 0, err
+ }
+
+ if uploadResult.Error != "" {
+ return 0, fmt.Errorf("upload error: %s", uploadResult.Error)
+ }
+
+ fid, err := needle.ParseFileIdFromString(assignResult.Fid)
+ if err != nil {
+ return 0, err
+ }
+
+ return fid.VolumeId, nil
+}
+
+// countShardsPerDisk counts EC shards on each disk of each server
+// Returns map: "serverN" -> map[diskId]shardCount
+func countShardsPerDisk(testDir string, volumeId uint32) map[string]map[int]int {
+ result := make(map[string]map[int]int)
+
+ const numServers = 3
+ const disksPerServer = 4
+
+ for server := 0; server < numServers; server++ {
+ serverKey := fmt.Sprintf("server%d", server)
+ result[serverKey] = make(map[int]int)
+
+ for disk := 0; disk < disksPerServer; disk++ {
+ diskDir := filepath.Join(testDir, fmt.Sprintf("server%d_disk%d", server, disk))
+ count, err := countECShardFiles(diskDir, volumeId)
+ if err == nil && count > 0 {
+ result[serverKey][disk] = count
+ }
+ }
+ }
+
+ return result
+}
+
+// calculateDiskShardVariance measures how evenly shards are distributed across disks
+// Lower variance means better distribution
+func calculateDiskShardVariance(distribution map[string]map[int]int) float64 {
+ var counts []float64
+
+ for _, disks := range distribution {
+ for _, count := range disks {
+ if count > 0 {
+ counts = append(counts, float64(count))
+ }
+ }
+ }
+
+ if len(counts) == 0 {
+ return 0
+ }
+
+ // Calculate mean
+ mean := 0.0
+ for _, c := range counts {
+ mean += c
+ }
+ mean /= float64(len(counts))
+
+ // Calculate variance
+ variance := 0.0
+ for _, c := range counts {
+ variance += (c - mean) * (c - mean)
+ }
+
+ return math.Sqrt(variance / float64(len(counts)))
+}
diff --git a/test/s3/sse/s3_sse_integration_test.go b/test/s3/sse/s3_sse_integration_test.go
index 7b939ea76..4b7eb0ddc 100644
--- a/test/s3/sse/s3_sse_integration_test.go
+++ b/test/s3/sse/s3_sse_integration_test.go
@@ -2082,6 +2082,78 @@ func TestCopyToBucketDefaultEncryptedRegression(t *testing.T) {
require.NoError(t, err, "Failed to read object")
assertDataEqual(t, testData, data, "Data mismatch")
})
+
+ t.Run("LargeFileCopyEncrypted_ToTemp_ToEncrypted", func(t *testing.T) {
+ // Test with large file (1MB) to exercise chunk-by-chunk copy path
+ // This verifies consistent behavior with SSE-C and SSE-KMS
+ largeTestData := generateTestData(1024 * 1024) // 1MB
+ objectKey := "large-file-test.bin"
+
+ // Step 1: Upload large object to source bucket (will be automatically encrypted)
+ _, err = client.PutObject(ctx, &s3.PutObjectInput{
+ Bucket: aws.String(srcBucket),
+ Key: aws.String(objectKey),
+ Body: bytes.NewReader(largeTestData),
+ })
+ require.NoError(t, err, "Failed to upload large file to source bucket")
+
+ // Verify source object is encrypted
+ srcHead, err := client.HeadObject(ctx, &s3.HeadObjectInput{
+ Bucket: aws.String(srcBucket),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Failed to HEAD source object")
+ assert.Equal(t, types.ServerSideEncryptionAes256, srcHead.ServerSideEncryption,
+ "Source object should be SSE-S3 encrypted")
+
+ // Step 2: Copy to temp bucket (unencrypted) - exercises chunk-by-chunk decrypt
+ _, err = client.CopyObject(ctx, &s3.CopyObjectInput{
+ Bucket: aws.String(tempBucket),
+ Key: aws.String(objectKey),
+ CopySource: aws.String(fmt.Sprintf("%s/%s", srcBucket, objectKey)),
+ })
+ require.NoError(t, err, "Failed to copy large file to temp bucket")
+
+ // Verify temp object is unencrypted and data is correct
+ tempGet, err := client.GetObject(ctx, &s3.GetObjectInput{
+ Bucket: aws.String(tempBucket),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Failed to GET temp object")
+ tempData, err := io.ReadAll(tempGet.Body)
+ tempGet.Body.Close()
+ require.NoError(t, err, "Failed to read temp object")
+ assertDataEqual(t, largeTestData, tempData, "Temp object data mismatch after decrypt")
+
+ // Step 3: Copy from temp bucket to dest bucket (with default encryption)
+ // This exercises chunk-by-chunk encrypt copy
+ _, err = client.CopyObject(ctx, &s3.CopyObjectInput{
+ Bucket: aws.String(dstBucket),
+ Key: aws.String(objectKey),
+ CopySource: aws.String(fmt.Sprintf("%s/%s", tempBucket, objectKey)),
+ })
+ require.NoError(t, err, "Failed to copy large file to destination bucket")
+
+ // Verify destination object is encrypted
+ dstHead, err := client.HeadObject(ctx, &s3.HeadObjectInput{
+ Bucket: aws.String(dstBucket),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Failed to HEAD destination object")
+ assert.Equal(t, types.ServerSideEncryptionAes256, dstHead.ServerSideEncryption,
+ "Destination object should be SSE-S3 encrypted via bucket default")
+
+ // Verify destination object content is correct after re-encryption
+ dstGet, err := client.GetObject(ctx, &s3.GetObjectInput{
+ Bucket: aws.String(dstBucket),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Failed to GET destination object")
+ dstData, err := io.ReadAll(dstGet.Body)
+ dstGet.Body.Close()
+ require.NoError(t, err, "Failed to read destination object")
+ assertDataEqual(t, largeTestData, dstData, "Large file data mismatch after re-encryption")
+ })
}
// REGRESSION TESTS FOR CRITICAL BUGS FIXED
diff --git a/test/s3/tagging/Makefile b/test/s3/tagging/Makefile
new file mode 100644
index 000000000..aa2f18f7c
--- /dev/null
+++ b/test/s3/tagging/Makefile
@@ -0,0 +1,321 @@
+# S3 Object Tagging Tests
+# Tests for GitHub issue #7589: S3 object Tags query comes back empty
+
+.PHONY: help build-weed setup-server start-server stop-server test-tagging test-tagging-quick test-tagging-comprehensive test-all clean logs check-deps test-with-server
+
+# Configuration
+WEED_BINARY := ../../../weed/weed_binary
+S3_PORT := 8006
+MASTER_PORT := 9338
+VOLUME_PORT := 8085
+FILER_PORT := 8893
+TEST_TIMEOUT := 10m
+TEST_PATTERN := TestObjectTaggingOnUpload|TestPutObjectTaggingAPI|TestDeleteObjectTagging|TestTag
+
+# Default target
+help:
+ @echo "S3 Object Tagging Tests Makefile"
+ @echo ""
+ @echo "Available targets:"
+ @echo " help - Show this help message"
+ @echo " build-weed - Build the SeaweedFS binary"
+ @echo " check-deps - Check dependencies and build binary if needed"
+ @echo " start-server - Start SeaweedFS server for testing"
+ @echo " stop-server - Stop SeaweedFS server"
+ @echo " test-tagging - Run all tagging tests"
+ @echo " test-tagging-quick - Run core tagging tests only"
+ @echo " test-tagging-comprehensive - Run comprehensive tagging tests"
+ @echo " test-with-server - Start server, run tests, stop server"
+ @echo " logs - Show server logs"
+ @echo " clean - Clean up test artifacts and stop server"
+ @echo " health-check - Check if server is accessible"
+ @echo ""
+ @echo "Configuration:"
+ @echo " S3_PORT=${S3_PORT}"
+ @echo " TEST_TIMEOUT=${TEST_TIMEOUT}"
+
+# Build the SeaweedFS binary
+build-weed:
+ @echo "Building SeaweedFS binary..."
+ @cd ../../../weed && go build -o weed_binary .
+ @chmod +x $(WEED_BINARY)
+ @echo "βœ… SeaweedFS binary built at $(WEED_BINARY)"
+
+check-deps: build-weed
+ @echo "Checking dependencies..."
+ @echo "πŸ” DEBUG: Checking Go installation..."
+ @command -v go >/dev/null 2>&1 || (echo "Go is required but not installed" && exit 1)
+ @echo "πŸ” DEBUG: Go version: $$(go version)"
+ @echo "πŸ” DEBUG: Checking binary at $(WEED_BINARY)..."
+ @test -f $(WEED_BINARY) || (echo "SeaweedFS binary not found at $(WEED_BINARY)" && exit 1)
+ @echo "πŸ” DEBUG: Binary size: $$(ls -lh $(WEED_BINARY) | awk '{print $$5}')"
+ @echo "πŸ” DEBUG: Binary permissions: $$(ls -la $(WEED_BINARY) | awk '{print $$1}')"
+ @echo "πŸ” DEBUG: Checking Go module dependencies..."
+ @go list -m github.com/aws/aws-sdk-go-v2 >/dev/null 2>&1 || (echo "AWS SDK Go v2 not found. Run 'go mod tidy'." && exit 1)
+ @go list -m github.com/stretchr/testify >/dev/null 2>&1 || (echo "Testify not found. Run 'go mod tidy'." && exit 1)
+ @echo "βœ… All dependencies are available"
+
+# Start SeaweedFS server for testing
+start-server: check-deps
+ @echo "Starting SeaweedFS server..."
+ @echo "πŸ” DEBUG: Current working directory: $$(pwd)"
+ @echo "πŸ” DEBUG: Checking for existing weed processes..."
+ @ps aux | grep weed | grep -v grep || echo "No existing weed processes found"
+ @echo "πŸ” DEBUG: Cleaning up any existing PID file..."
+ @rm -f weed-server.pid
+ @echo "πŸ” DEBUG: Checking for port conflicts..."
+ @if netstat -tlnp 2>/dev/null | grep $(S3_PORT) >/dev/null; then \
+ echo "⚠️ Port $(S3_PORT) is already in use, trying to find the process..."; \
+ netstat -tlnp 2>/dev/null | grep $(S3_PORT) || true; \
+ else \
+ echo "βœ… Port $(S3_PORT) is available"; \
+ fi
+ @echo "πŸ” DEBUG: Checking binary at $(WEED_BINARY)"
+ @ls -la $(WEED_BINARY) || (echo "❌ Binary not found!" && exit 1)
+ @echo "πŸ” DEBUG: Checking config file at ../../../docker/compose/s3.json"
+ @ls -la ../../../docker/compose/s3.json || echo "⚠️ Config file not found, continuing without it"
+ @echo "πŸ” DEBUG: Creating volume directory..."
+ @mkdir -p ./test-volume-data
+ @echo "πŸ” DEBUG: Launching SeaweedFS server in background..."
+ @echo "πŸ” DEBUG: Command: $(WEED_BINARY) server -filer -filer.maxMB=64 -s3 -ip.bind 0.0.0.0 -dir=./test-volume-data -master.raftHashicorp -master.electionTimeout 1s -master.volumeSizeLimitMB=100 -volume.max=100 -volume.preStopSeconds=1 -master.port=$(MASTER_PORT) -volume.port=$(VOLUME_PORT) -filer.port=$(FILER_PORT) -s3.port=$(S3_PORT) -metricsPort=9329 -s3.allowEmptyFolder=false -s3.allowDeleteBucketNotEmpty=true -s3.config=../../../docker/compose/s3.json -master.peers=none"
+ @$(WEED_BINARY) server \
+ -filer \
+ -filer.maxMB=64 \
+ -s3 \
+ -ip.bind 0.0.0.0 \
+ -dir=./test-volume-data \
+ -master.raftHashicorp \
+ -master.electionTimeout 1s \
+ -master.volumeSizeLimitMB=100 \
+ -volume.max=100 \
+ -volume.preStopSeconds=1 \
+ -master.port=$(MASTER_PORT) \
+ -volume.port=$(VOLUME_PORT) \
+ -filer.port=$(FILER_PORT) \
+ -s3.port=$(S3_PORT) \
+ -metricsPort=9329 \
+ -s3.allowEmptyFolder=false \
+ -s3.allowDeleteBucketNotEmpty=true \
+ -s3.config=../../../docker/compose/s3.json \
+ -master.peers=none \
+ > weed-test.log 2>&1 & echo $$! > weed-server.pid
+ @echo "πŸ” DEBUG: Server PID: $$(cat weed-server.pid 2>/dev/null || echo 'PID file not found')"
+ @echo "πŸ” DEBUG: Checking if PID is still running..."
+ @sleep 2
+ @if [ -f weed-server.pid ]; then \
+ SERVER_PID=$$(cat weed-server.pid); \
+ ps -p $$SERVER_PID || echo "⚠️ Server PID $$SERVER_PID not found after 2 seconds"; \
+ else \
+ echo "⚠️ PID file not found"; \
+ fi
+ @echo "πŸ” DEBUG: Waiting for server to start (up to 90 seconds)..."
+ @for i in $$(seq 1 90); do \
+ echo "πŸ” DEBUG: Attempt $$i/90 - checking port $(S3_PORT)"; \
+ if curl -s http://localhost:$(S3_PORT) >/dev/null 2>&1; then \
+ echo "βœ… SeaweedFS server started successfully on port $(S3_PORT) after $$i seconds"; \
+ exit 0; \
+ fi; \
+ if [ $$i -eq 5 ]; then \
+ echo "πŸ” DEBUG: After 5 seconds, checking process and logs..."; \
+ ps aux | grep weed | grep -v grep || echo "No weed processes found"; \
+ if [ -f weed-test.log ]; then \
+ echo "=== First server logs ==="; \
+ head -20 weed-test.log; \
+ fi; \
+ fi; \
+ if [ $$i -eq 15 ]; then \
+ echo "πŸ” DEBUG: After 15 seconds, checking port bindings..."; \
+ netstat -tlnp 2>/dev/null | grep $(S3_PORT) || echo "Port $(S3_PORT) not bound"; \
+ netstat -tlnp 2>/dev/null | grep $(MASTER_PORT) || echo "Port $(MASTER_PORT) not bound"; \
+ netstat -tlnp 2>/dev/null | grep $(VOLUME_PORT) || echo "Port $(VOLUME_PORT) not bound"; \
+ fi; \
+ if [ $$i -eq 30 ]; then \
+ echo "⚠️ Server taking longer than expected (30s), checking logs..."; \
+ if [ -f weed-test.log ]; then \
+ echo "=== Recent server logs ==="; \
+ tail -20 weed-test.log; \
+ fi; \
+ fi; \
+ sleep 1; \
+ done; \
+ echo "❌ Server failed to start within 90 seconds"; \
+ echo "πŸ” DEBUG: Final process check:"; \
+ ps aux | grep weed | grep -v grep || echo "No weed processes found"; \
+ echo "πŸ” DEBUG: Final port check:"; \
+ netstat -tlnp 2>/dev/null | grep -E "($(S3_PORT)|$(MASTER_PORT)|$(VOLUME_PORT))" || echo "No ports bound"; \
+ echo "=== Full server logs ==="; \
+ if [ -f weed-test.log ]; then \
+ cat weed-test.log; \
+ else \
+ echo "No log file found"; \
+ fi; \
+ exit 1
+
+# Stop SeaweedFS server
+stop-server:
+ @echo "Stopping SeaweedFS server..."
+ @if [ -f weed-server.pid ]; then \
+ SERVER_PID=$$(cat weed-server.pid); \
+ echo "Killing server PID $$SERVER_PID"; \
+ if ps -p $$SERVER_PID >/dev/null 2>&1; then \
+ kill -TERM $$SERVER_PID 2>/dev/null || true; \
+ sleep 2; \
+ if ps -p $$SERVER_PID >/dev/null 2>&1; then \
+ echo "Process still running, sending KILL signal..."; \
+ kill -KILL $$SERVER_PID 2>/dev/null || true; \
+ sleep 1; \
+ fi; \
+ else \
+ echo "Process $$SERVER_PID not found (already stopped)"; \
+ fi; \
+ rm -f weed-server.pid; \
+ else \
+ echo "No PID file found, checking for running processes..."; \
+ echo "⚠️ Skipping automatic process cleanup to avoid CI issues"; \
+ echo "Note: Any remaining weed processes should be cleaned up by the CI environment"; \
+ fi
+ @echo "βœ… SeaweedFS server stopped"
+
+# Show server logs
+logs:
+ @if test -f weed-test.log; then \
+ echo "=== SeaweedFS Server Logs ==="; \
+ tail -f weed-test.log; \
+ else \
+ echo "No log file found. Server may not be running."; \
+ fi
+
+# Core tagging tests (basic functionality)
+test-tagging-quick: check-deps
+ @echo "Running core tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "TestObjectTaggingOnUpload|TestPutObjectTaggingAPI" .
+ @echo "βœ… Core tagging tests completed"
+
+# All tagging tests (comprehensive)
+test-tagging: check-deps
+ @echo "Running all tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "$(TEST_PATTERN)" .
+ @echo "βœ… All tagging tests completed"
+
+# Comprehensive tagging tests (all features)
+test-tagging-comprehensive: check-deps
+ @echo "Running comprehensive tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) .
+ @echo "βœ… Comprehensive tagging tests completed"
+
+# All tests without server management
+test-tagging-simple: check-deps
+ @echo "Running tagging tests (assuming server is already running)..."
+ @go test -v -timeout=$(TEST_TIMEOUT) .
+ @echo "βœ… All tagging tests completed"
+
+# Start server, run tests, stop server
+test-with-server: start-server
+ @echo "Running tagging tests with managed server..."
+ @sleep 5 # Give server time to fully start
+ @make test-tagging-comprehensive || (echo "Tests failed, stopping server..." && make stop-server && exit 1)
+ @make stop-server
+ @echo "βœ… All tests completed with managed server"
+
+# Health check
+health-check:
+ @echo "Checking server health..."
+ @if curl -s http://localhost:$(S3_PORT) >/dev/null 2>&1; then \
+ echo "βœ… Server is accessible on port $(S3_PORT)"; \
+ else \
+ echo "❌ Server is not accessible on port $(S3_PORT)"; \
+ exit 1; \
+ fi
+
+# Clean up
+clean:
+ @echo "Cleaning up test artifacts..."
+ @make stop-server
+ @rm -f weed-test.log
+ @rm -f weed-server.pid
+ @rm -rf ./test-volume-data
+ @rm -f tagging.test
+ @go clean -testcache
+ @echo "βœ… Cleanup completed"
+
+# Individual test targets for specific functionality
+test-upload:
+ @echo "Running upload tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "TestObjectTaggingOnUpload" .
+
+test-special-chars:
+ @echo "Running special characters tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "TestObjectTaggingOnUploadWithSpecialCharacters" .
+
+test-api:
+ @echo "Running API tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "TestPutObjectTaggingAPI" .
+
+test-get:
+ @echo "Running get tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "TestGetObjectTaggingAPI" .
+
+test-delete:
+ @echo "Running delete tagging tests..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "TestDeleteObjectTaggingAPI" .
+
+# Development targets
+dev-start: start-server
+ @echo "Development server started. Access S3 API at http://localhost:$(S3_PORT)"
+ @echo "To stop: make stop-server"
+
+dev-test: check-deps
+ @echo "Running tests in development mode..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -run "TestObjectTaggingOnUpload" .
+
+# CI targets
+ci-test: check-deps
+ @echo "Running tests in CI mode..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -race .
+
+# All targets
+test-all: test-tagging test-tagging-comprehensive
+ @echo "βœ… All tagging tests completed"
+
+# Benchmark targets
+benchmark-tagging:
+ @echo "Running tagging performance benchmarks..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -bench=. -benchmem .
+
+# Coverage targets
+coverage:
+ @echo "Running tests with coverage..."
+ @go test -v -timeout=$(TEST_TIMEOUT) -coverprofile=coverage.out .
+ @go tool cover -html=coverage.out -o coverage.html
+ @echo "Coverage report generated: coverage.html"
+
+# Format and lint
+fmt:
+ @echo "Formatting Go code..."
+ @go fmt .
+
+lint:
+ @echo "Running linter..."
+ @golint . || echo "golint not available, skipping..."
+
+# Install dependencies for development
+install-deps:
+ @echo "Installing Go dependencies..."
+ @go mod tidy
+ @go mod download
+
+# Show current configuration
+show-config:
+ @echo "Current configuration:"
+ @echo " WEED_BINARY: $(WEED_BINARY)"
+ @echo " S3_PORT: $(S3_PORT)"
+ @echo " TEST_TIMEOUT: $(TEST_TIMEOUT)"
+ @echo " TEST_PATTERN: $(TEST_PATTERN)"
+
+# Legacy targets for backward compatibility
+test: test-with-server
+test-verbose: test-tagging-comprehensive
+test-single: test-upload
+test-clean: clean
+build: check-deps
+setup: check-deps
diff --git a/test/s3/tagging/README.md b/test/s3/tagging/README.md
new file mode 100644
index 000000000..73f2a46c4
--- /dev/null
+++ b/test/s3/tagging/README.md
@@ -0,0 +1,53 @@
+# S3 Object Tagging Tests
+
+This directory contains tests for S3 object tagging functionality.
+
+## Issue Reference
+
+These tests were created to verify the fix for [GitHub Issue #7589](https://github.com/seaweedfs/seaweedfs/issues/7589):
+**S3 object Tags query comes back empty**
+
+## Problem Description
+
+When uploading an object with tags using the `X-Amz-Tagging` header, the tags were not being stored.
+When querying the object tagging with `GetObjectTagging`, the response was empty.
+
+This was a regression between SeaweedFS 4.00 and 4.01.
+
+## Root Cause
+
+The `putToFiler` function in `s3api_object_handlers_put.go` was not parsing the `X-Amz-Tagging` header
+and storing the tags in the entry's Extended metadata. The code was only copying user metadata
+(headers starting with `X-Amz-Meta-`) but not object tags.
+
+## Fix
+
+Added tag parsing logic to `putToFiler` that:
+1. Reads the `X-Amz-Tagging` header
+2. Parses it using `url.ParseQuery()` for proper URL decoding
+3. Stores each tag with the prefix `X-Amz-Tagging-` in the entry's Extended metadata
+
+## Running Tests
+
+```bash
+# Run all tagging tests
+cd test/s3/tagging
+make test
+
+# Run specific test
+make test-upload
+
+# Or using go test directly
+go test -v ./...
+```
+
+## Test Cases
+
+1. **TestObjectTaggingOnUpload** - Basic test for tags sent during object upload
+2. **TestObjectTaggingOnUploadWithSpecialCharacters** - Tests URL-encoded tag values
+3. **TestObjectTaggingOnUploadWithEmptyValue** - Tests tags with empty values
+4. **TestPutObjectTaggingAPI** - Tests the PutObjectTagging API separately
+5. **TestDeleteObjectTagging** - Tests tag deletion
+6. **TestTagsNotPreservedAfterObjectOverwrite** - Verifies AWS S3 behavior on overwrite
+7. **TestMaximumNumberOfTags** - Tests storing the maximum 10 tags
+8. **TestTagCountHeader** - Tests the x-amz-tagging-count header in HeadObject
diff --git a/test/s3/tagging/s3_tagging_test.go b/test/s3/tagging/s3_tagging_test.go
new file mode 100644
index 000000000..c490ca1aa
--- /dev/null
+++ b/test/s3/tagging/s3_tagging_test.go
@@ -0,0 +1,446 @@
+package tagging
+
+import (
+ "context"
+ "fmt"
+ "os"
+ "strings"
+ "testing"
+ "time"
+
+ "github.com/aws/aws-sdk-go-v2/aws"
+ "github.com/aws/aws-sdk-go-v2/config"
+ "github.com/aws/aws-sdk-go-v2/credentials"
+ "github.com/aws/aws-sdk-go-v2/service/s3"
+ "github.com/aws/aws-sdk-go-v2/service/s3/types"
+ "github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+)
+
+// S3TestConfig holds configuration for S3 tests
+type S3TestConfig struct {
+ Endpoint string
+ AccessKey string
+ SecretKey string
+ Region string
+ BucketPrefix string
+ UseSSL bool
+ SkipVerifySSL bool
+}
+
+// getDefaultConfig returns a fresh instance of the default test configuration
+func getDefaultConfig() *S3TestConfig {
+ endpoint := os.Getenv("S3_ENDPOINT")
+ if endpoint == "" {
+ endpoint = "http://localhost:8333" // Default SeaweedFS S3 port
+ }
+ accessKey := os.Getenv("S3_ACCESS_KEY")
+ if accessKey == "" {
+ accessKey = "some_access_key1"
+ }
+ secretKey := os.Getenv("S3_SECRET_KEY")
+ if secretKey == "" {
+ secretKey = "some_secret_key1"
+ }
+ return &S3TestConfig{
+ Endpoint: endpoint,
+ AccessKey: accessKey,
+ SecretKey: secretKey,
+ Region: "us-east-1",
+ BucketPrefix: "test-tagging-",
+ UseSSL: false,
+ SkipVerifySSL: true,
+ }
+}
+
+// getS3Client creates an AWS S3 client for testing
+func getS3Client(t *testing.T) *s3.Client {
+ defaultConfig := getDefaultConfig()
+ cfg, err := config.LoadDefaultConfig(context.TODO(),
+ config.WithRegion(defaultConfig.Region),
+ config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider(
+ defaultConfig.AccessKey,
+ defaultConfig.SecretKey,
+ "",
+ )),
+ config.WithEndpointResolverWithOptions(aws.EndpointResolverWithOptionsFunc(
+ func(service, region string, options ...interface{}) (aws.Endpoint, error) {
+ return aws.Endpoint{
+ URL: defaultConfig.Endpoint,
+ SigningRegion: defaultConfig.Region,
+ }, nil
+ })),
+ )
+ require.NoError(t, err)
+
+ client := s3.NewFromConfig(cfg, func(o *s3.Options) {
+ o.UsePathStyle = true
+ })
+ return client
+}
+
+// createTestBucket creates a test bucket with a unique name
+func createTestBucket(t *testing.T, client *s3.Client) string {
+ defaultConfig := getDefaultConfig()
+ bucketName := fmt.Sprintf("%s%d", defaultConfig.BucketPrefix, time.Now().UnixNano())
+
+ _, err := client.CreateBucket(context.TODO(), &s3.CreateBucketInput{
+ Bucket: aws.String(bucketName),
+ })
+ require.NoError(t, err)
+
+ // Wait for bucket metadata to be fully processed
+ time.Sleep(50 * time.Millisecond)
+
+ return bucketName
+}
+
+// cleanupTestBucket removes the test bucket and all its contents
+func cleanupTestBucket(t *testing.T, client *s3.Client, bucketName string) {
+ // First, delete all objects in the bucket
+ listResp, err := client.ListObjectsV2(context.TODO(), &s3.ListObjectsV2Input{
+ Bucket: aws.String(bucketName),
+ })
+ if err == nil {
+ for _, obj := range listResp.Contents {
+ _, err := client.DeleteObject(context.TODO(), &s3.DeleteObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: obj.Key,
+ })
+ if err != nil {
+ t.Logf("Warning: failed to delete object %s: %v", *obj.Key, err)
+ }
+ }
+ }
+
+ // Then delete the bucket
+ _, err = client.DeleteBucket(context.TODO(), &s3.DeleteBucketInput{
+ Bucket: aws.String(bucketName),
+ })
+ if err != nil {
+ t.Logf("Warning: failed to delete bucket %s: %v", bucketName, err)
+ }
+}
+
+// TestObjectTaggingOnUpload tests that tags sent during object upload (via X-Amz-Tagging header)
+// are properly stored and can be retrieved. This is the fix for GitHub issue #7589.
+func TestObjectTaggingOnUpload(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-with-tags"
+ objectContent := "Hello, World!"
+
+ // Put object with tags using the Tagging parameter (X-Amz-Tagging header)
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ Tagging: aws.String("env=production&team=platform"),
+ })
+ require.NoError(t, err, "Should be able to put object with tags")
+
+ // Get the tags back
+ tagResp, err := client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags")
+
+ // Verify tags were stored correctly
+ require.Len(t, tagResp.TagSet, 2, "Should have 2 tags")
+
+ // Build a map for easier assertion
+ tagMap := make(map[string]string)
+ for _, tag := range tagResp.TagSet {
+ tagMap[*tag.Key] = *tag.Value
+ }
+
+ assert.Equal(t, "production", tagMap["env"], "env tag should be 'production'")
+ assert.Equal(t, "platform", tagMap["team"], "team tag should be 'platform'")
+}
+
+// TestObjectTaggingOnUploadWithSpecialCharacters tests tags with URL-encoded characters
+func TestObjectTaggingOnUploadWithSpecialCharacters(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-with-special-tags"
+ objectContent := "Hello, World!"
+
+ // Put object with tags containing special characters
+ // AWS SDK will URL-encode these automatically
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ Tagging: aws.String("timestamp=2025-07-16 14:40:39&path=/tmp/file.txt"),
+ })
+ require.NoError(t, err, "Should be able to put object with special character tags")
+
+ // Get the tags back
+ tagResp, err := client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags")
+
+ // Verify tags were stored and URL-decoded correctly
+ require.Len(t, tagResp.TagSet, 2, "Should have 2 tags")
+
+ tagMap := make(map[string]string)
+ for _, tag := range tagResp.TagSet {
+ tagMap[*tag.Key] = *tag.Value
+ }
+
+ assert.Equal(t, "2025-07-16 14:40:39", tagMap["timestamp"], "timestamp tag should be decoded correctly")
+ assert.Equal(t, "/tmp/file.txt", tagMap["path"], "path tag should be decoded correctly")
+}
+
+// TestObjectTaggingOnUploadWithEmptyValue tests tags with empty values
+func TestObjectTaggingOnUploadWithEmptyValue(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-with-empty-tag"
+ objectContent := "Hello, World!"
+
+ // Put object with a tag that has an empty value
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ Tagging: aws.String("marker=&env=dev"),
+ })
+ require.NoError(t, err, "Should be able to put object with empty tag value")
+
+ // Get the tags back
+ tagResp, err := client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags")
+
+ // Verify tags were stored correctly
+ require.Len(t, tagResp.TagSet, 2, "Should have 2 tags")
+
+ tagMap := make(map[string]string)
+ for _, tag := range tagResp.TagSet {
+ tagMap[*tag.Key] = *tag.Value
+ }
+
+ assert.Equal(t, "", tagMap["marker"], "marker tag should have empty value")
+ assert.Equal(t, "dev", tagMap["env"], "env tag should be 'dev'")
+}
+
+// TestPutObjectTaggingAPI tests the PutObjectTagging API separately from upload
+func TestPutObjectTaggingAPI(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-for-tagging-api"
+ objectContent := "Hello, World!"
+
+ // First, put object without tags
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ })
+ require.NoError(t, err, "Should be able to put object without tags")
+
+ // Get tags - should be empty
+ tagResp, err := client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags")
+ assert.Len(t, tagResp.TagSet, 0, "Should have no tags initially")
+
+ // Now add tags using PutObjectTagging API
+ _, err = client.PutObjectTagging(context.TODO(), &s3.PutObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Tagging: &types.Tagging{
+ TagSet: []types.Tag{
+ {Key: aws.String("env"), Value: aws.String("staging")},
+ {Key: aws.String("version"), Value: aws.String("1.0")},
+ },
+ },
+ })
+ require.NoError(t, err, "Should be able to put object tags via API")
+
+ // Get tags - should now have the tags
+ tagResp, err = client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags after PutObjectTagging")
+ require.Len(t, tagResp.TagSet, 2, "Should have 2 tags")
+
+ tagMap := make(map[string]string)
+ for _, tag := range tagResp.TagSet {
+ tagMap[*tag.Key] = *tag.Value
+ }
+
+ assert.Equal(t, "staging", tagMap["env"], "env tag should be 'staging'")
+ assert.Equal(t, "1.0", tagMap["version"], "version tag should be '1.0'")
+}
+
+// TestDeleteObjectTagging tests the DeleteObjectTagging API
+func TestDeleteObjectTagging(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-for-delete-tags"
+ objectContent := "Hello, World!"
+
+ // Put object with tags
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ Tagging: aws.String("env=production"),
+ })
+ require.NoError(t, err, "Should be able to put object with tags")
+
+ // Verify tags exist
+ tagResp, err := client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags")
+ require.Len(t, tagResp.TagSet, 1, "Should have 1 tag")
+
+ // Delete tags
+ _, err = client.DeleteObjectTagging(context.TODO(), &s3.DeleteObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to delete object tags")
+
+ // Verify tags are deleted
+ tagResp, err = client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags after deletion")
+ assert.Len(t, tagResp.TagSet, 0, "Should have no tags after deletion")
+}
+
+// TestTagsNotPreservedAfterObjectOverwrite tests that tags are NOT preserved when an object is overwritten
+// This matches AWS S3 behavior where overwriting an object replaces all metadata including tags
+func TestTagsNotPreservedAfterObjectOverwrite(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-overwrite-tags"
+ objectContent := "Original content"
+
+ // Put object with tags
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ Tagging: aws.String("original=true"),
+ })
+ require.NoError(t, err, "Should be able to put object with tags")
+
+ // Verify original tags exist
+ tagResp, err := client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags")
+ require.Len(t, tagResp.TagSet, 1, "Should have 1 tag")
+ assert.Equal(t, "original", *tagResp.TagSet[0].Key)
+
+ // Overwrite the object WITHOUT tags
+ _, err = client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader("New content"),
+ })
+ require.NoError(t, err, "Should be able to overwrite object")
+
+ // Tags should be gone after overwrite (matches AWS S3 behavior)
+ tagResp, err = client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags after overwrite")
+ assert.Len(t, tagResp.TagSet, 0, "Tags should be cleared after object overwrite")
+}
+
+// TestMaximumNumberOfTags tests that we can store the maximum 10 tags per object
+func TestMaximumNumberOfTags(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-max-tags"
+ objectContent := "Hello, World!"
+
+ // Build 10 tags (S3 max)
+ tags := []string{}
+ for i := 1; i <= 10; i++ {
+ tags = append(tags, fmt.Sprintf("key%d=value%d", i, i))
+ }
+ tagging := strings.Join(tags, "&")
+
+ // Put object with 10 tags
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ Tagging: aws.String(tagging),
+ })
+ require.NoError(t, err, "Should be able to put object with 10 tags")
+
+ // Get the tags back
+ tagResp, err := client.GetObjectTagging(context.TODO(), &s3.GetObjectTaggingInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to get object tags")
+ assert.Len(t, tagResp.TagSet, 10, "Should have 10 tags")
+}
+
+// TestTagCountHeader tests that the x-amz-tagging-count header is returned in HeadObject
+func TestTagCountHeader(t *testing.T) {
+ client := getS3Client(t)
+ bucketName := createTestBucket(t, client)
+ defer cleanupTestBucket(t, client, bucketName)
+
+ objectKey := "test-object-tag-count"
+ objectContent := "Hello, World!"
+
+ // Put object with tags
+ _, err := client.PutObject(context.TODO(), &s3.PutObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ Body: strings.NewReader(objectContent),
+ Tagging: aws.String("env=prod&team=backend&version=2.0"),
+ })
+ require.NoError(t, err, "Should be able to put object with tags")
+
+ // Head object to get tag count
+ headResp, err := client.HeadObject(context.TODO(), &s3.HeadObjectInput{
+ Bucket: aws.String(bucketName),
+ Key: aws.String(objectKey),
+ })
+ require.NoError(t, err, "Should be able to head object")
+
+ // Check tag count header
+ if headResp.TagCount != nil {
+ assert.Equal(t, int32(3), *headResp.TagCount, "Tag count should be 3")
+ } else {
+ t.Log("Warning: TagCount header not returned - this may be expected depending on implementation")
+ }
+}
diff --git a/test/sftp/Makefile b/test/sftp/Makefile
new file mode 100644
index 000000000..bc46dd3ce
--- /dev/null
+++ b/test/sftp/Makefile
@@ -0,0 +1,41 @@
+.PHONY: all build test test-verbose test-short test-homedir test-debug clean deps tidy
+
+all: build test
+
+# Build the weed binary first
+build:
+ cd ../../weed && go build -o weed .
+
+# Install test dependencies
+deps:
+ go mod download
+
+# Run all tests
+test: build deps
+ go test -timeout 5m ./...
+
+# Run tests with verbose output
+test-verbose: build deps
+ go test -v -timeout 5m ./...
+
+# Run quick tests only (skip integration tests)
+test-short: deps
+ go test -short -v ./...
+
+# Run specific test
+test-homedir: build deps
+ go test -v -timeout 5m -run TestHomeDirPathTranslation ./...
+
+# Run tests with debug output from SeaweedFS
+test-debug: build deps
+ go test -v -timeout 5m ./... 2>&1 | tee test.log
+
+# Clean up test artifacts
+clean:
+ rm -f test.log
+ go clean -testcache
+
+# Update go.sum
+tidy:
+ go mod tidy
+
diff --git a/test/sftp/README.md b/test/sftp/README.md
new file mode 100644
index 000000000..17b5e67c7
--- /dev/null
+++ b/test/sftp/README.md
@@ -0,0 +1,92 @@
+# SeaweedFS SFTP Integration Tests
+
+This directory contains integration tests for the SeaweedFS SFTP server.
+
+## Prerequisites
+
+1. Build the SeaweedFS binary:
+ ```bash
+ cd ../../weed
+ go build -o weed .
+ ```
+
+2. Ensure `ssh-keygen` is available (for generating test SSH host keys)
+
+## Running Tests
+
+### Run all tests
+```bash
+make test
+```
+
+### Run tests with verbose output
+```bash
+make test-verbose
+```
+
+### Run a specific test
+```bash
+go test -v -run TestHomeDirPathTranslation
+```
+
+### Skip long-running tests
+```bash
+go test -short ./...
+```
+
+## Test Structure
+
+- `framework.go` - Test framework that starts SeaweedFS cluster with SFTP
+- `basic_test.go` - Basic SFTP operation tests including:
+ - HomeDir path translation (fixes issue #7470)
+ - File upload/download
+ - Directory operations
+ - Large file handling
+ - Edge cases
+
+## Test Configuration
+
+Tests use `testdata/userstore.json` which defines test users:
+
+| Username | Password | HomeDir | Permissions |
+|----------|----------|---------|-------------|
+| admin | adminpassword | / | Full access |
+| testuser | testuserpassword | /sftp/testuser | Full access to home |
+| readonly | readonlypassword | /public | Read-only |
+
+## Key Tests
+
+### TestHomeDirPathTranslation
+
+Tests the fix for [issue #7470](https://github.com/seaweedfs/seaweedfs/issues/7470) where
+users with a non-root HomeDir (e.g., `/sftp/testuser`) could not upload files to `/`
+because the path wasn't being translated to their home directory.
+
+The test verifies:
+- Uploading to `/` correctly maps to the user's HomeDir
+- Creating directories at `/` works
+- Listing `/` shows the user's home directory contents
+- All path operations respect the HomeDir translation
+
+## Debugging
+
+To debug test failures:
+
+1. Enable verbose output:
+ ```bash
+ go test -v -run TestName
+ ```
+
+2. Keep test artifacts (don't cleanup):
+ ```go
+ config := DefaultTestConfig()
+ config.SkipCleanup = true
+ ```
+
+3. Enable debug logging:
+ ```go
+ config := DefaultTestConfig()
+ config.EnableDebug = true
+ ```
+
+
diff --git a/test/sftp/basic_test.go b/test/sftp/basic_test.go
new file mode 100644
index 000000000..e5ffe90d1
--- /dev/null
+++ b/test/sftp/basic_test.go
@@ -0,0 +1,652 @@
+package sftp
+
+import (
+ "bytes"
+ "io"
+ "path"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+)
+
+// TestHomeDirPathTranslation tests that SFTP operations correctly translate
+// paths relative to the user's HomeDir.
+// This is the fix for https://github.com/seaweedfs/seaweedfs/issues/7470
+func TestHomeDirPathTranslation(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ // Test with user "testuser" who has HomeDir="/sftp/testuser"
+ // When they upload to "/", it should actually go to "/sftp/testuser"
+ sftpClient, sshConn, err := fw.ConnectSFTP("testuser", "testuserpassword")
+ require.NoError(t, err, "failed to connect as testuser")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ // Test 1: Upload file to "/" (should map to /sftp/testuser/)
+ t.Run("UploadToRoot", func(t *testing.T) {
+ testContent := []byte("Hello from SFTP test!")
+ filename := "test_upload.txt"
+
+ // Create file at "/" from user's perspective
+ file, err := sftpClient.Create("/" + filename)
+ require.NoError(t, err, "should be able to create file at /")
+
+ _, err = file.Write(testContent)
+ require.NoError(t, err, "should be able to write to file")
+ err = file.Close()
+ require.NoError(t, err, "should be able to close file")
+
+ // Verify file exists and has correct content
+ readFile, err := sftpClient.Open("/" + filename)
+ require.NoError(t, err, "should be able to open file")
+ defer readFile.Close()
+
+ content, err := io.ReadAll(readFile)
+ require.NoError(t, err, "should be able to read file")
+ require.Equal(t, testContent, content, "file content should match")
+
+ // Clean up
+ err = sftpClient.Remove("/" + filename)
+ require.NoError(t, err, "should be able to remove file")
+ })
+
+ // Test 2: Create directory at "/" (should map to /sftp/testuser/)
+ t.Run("CreateDirAtRoot", func(t *testing.T) {
+ dirname := "test_dir"
+
+ err := sftpClient.Mkdir("/" + dirname)
+ require.NoError(t, err, "should be able to create directory at /")
+
+ // Verify directory exists
+ info, err := sftpClient.Stat("/" + dirname)
+ require.NoError(t, err, "should be able to stat directory")
+ require.True(t, info.IsDir(), "should be a directory")
+
+ // Clean up
+ err = sftpClient.RemoveDirectory("/" + dirname)
+ require.NoError(t, err, "should be able to remove directory")
+ })
+
+ // Test 3: List directory at "/" (should list /sftp/testuser/)
+ t.Run("ListRoot", func(t *testing.T) {
+ // Create a test file first
+ testContent := []byte("list test content")
+ filename := "list_test.txt"
+
+ file, err := sftpClient.Create("/" + filename)
+ require.NoError(t, err)
+ _, err = file.Write(testContent)
+ require.NoError(t, err)
+ file.Close()
+
+ // List root directory
+ files, err := sftpClient.ReadDir("/")
+ require.NoError(t, err, "should be able to list root directory")
+
+ // Should find our test file
+ found := false
+ for _, f := range files {
+ if f.Name() == filename {
+ found = true
+ break
+ }
+ }
+ require.True(t, found, "should find test file in listing")
+
+ // Clean up
+ err = sftpClient.Remove("/" + filename)
+ require.NoError(t, err)
+ })
+
+ // Test 4: Nested directory operations
+ t.Run("NestedOperations", func(t *testing.T) {
+ // Create nested directory structure
+ err := sftpClient.MkdirAll("/nested/dir/structure")
+ require.NoError(t, err, "should be able to create nested directories")
+
+ // Create file in nested directory
+ testContent := []byte("nested file content")
+ file, err := sftpClient.Create("/nested/dir/structure/file.txt")
+ require.NoError(t, err)
+ _, err = file.Write(testContent)
+ require.NoError(t, err)
+ file.Close()
+
+ // Verify file exists
+ readFile, err := sftpClient.Open("/nested/dir/structure/file.txt")
+ require.NoError(t, err)
+ content, err := io.ReadAll(readFile)
+ require.NoError(t, err)
+ readFile.Close()
+ require.Equal(t, testContent, content)
+
+ // Clean up
+ err = sftpClient.Remove("/nested/dir/structure/file.txt")
+ require.NoError(t, err)
+ err = sftpClient.RemoveDirectory("/nested/dir/structure")
+ require.NoError(t, err)
+ err = sftpClient.RemoveDirectory("/nested/dir")
+ require.NoError(t, err)
+ err = sftpClient.RemoveDirectory("/nested")
+ require.NoError(t, err)
+ })
+
+ // Test 5: Rename operation
+ t.Run("RenameFile", func(t *testing.T) {
+ testContent := []byte("rename test content")
+
+ file, err := sftpClient.Create("/original.txt")
+ require.NoError(t, err)
+ _, err = file.Write(testContent)
+ require.NoError(t, err)
+ file.Close()
+
+ // Rename file
+ err = sftpClient.Rename("/original.txt", "/renamed.txt")
+ require.NoError(t, err, "should be able to rename file")
+
+ // Verify old file doesn't exist
+ _, err = sftpClient.Stat("/original.txt")
+ require.Error(t, err, "original file should not exist")
+
+ // Verify new file exists with correct content
+ readFile, err := sftpClient.Open("/renamed.txt")
+ require.NoError(t, err, "renamed file should exist")
+ content, err := io.ReadAll(readFile)
+ require.NoError(t, err)
+ readFile.Close()
+ require.Equal(t, testContent, content)
+
+ // Clean up
+ err = sftpClient.Remove("/renamed.txt")
+ require.NoError(t, err)
+ })
+}
+
+// TestAdminRootAccess tests that admin user with HomeDir="/" can access everything
+func TestAdminRootAccess(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ // Connect as admin with HomeDir="/"
+ sftpClient, sshConn, err := fw.ConnectSFTP("admin", "adminpassword")
+ require.NoError(t, err, "failed to connect as admin")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ // Admin should be able to create directories anywhere
+ t.Run("CreateAnyDirectory", func(t *testing.T) {
+ // Create the user's home directory structure
+ err := sftpClient.MkdirAll("/sftp/testuser")
+ require.NoError(t, err, "admin should be able to create any directory")
+
+ // Create file in that directory
+ testContent := []byte("admin created this")
+ file, err := sftpClient.Create("/sftp/testuser/admin_file.txt")
+ require.NoError(t, err)
+ _, err = file.Write(testContent)
+ require.NoError(t, err)
+ file.Close()
+
+ // Verify file exists
+ info, err := sftpClient.Stat("/sftp/testuser/admin_file.txt")
+ require.NoError(t, err)
+ require.False(t, info.IsDir())
+
+ // Clean up
+ err = sftpClient.Remove("/sftp/testuser/admin_file.txt")
+ require.NoError(t, err)
+ })
+}
+
+// TestLargeFileUpload tests uploading larger files through SFTP
+func TestLargeFileUpload(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ sftpClient, sshConn, err := fw.ConnectSFTP("testuser", "testuserpassword")
+ require.NoError(t, err, "failed to connect as testuser")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ // Create a 1MB file
+ t.Run("Upload1MB", func(t *testing.T) {
+ size := 1024 * 1024 // 1MB
+ testData := bytes.Repeat([]byte("A"), size)
+
+ file, err := sftpClient.Create("/large_file.bin")
+ require.NoError(t, err)
+ n, err := file.Write(testData)
+ require.NoError(t, err)
+ require.Equal(t, size, n)
+ file.Close()
+
+ // Verify file size
+ info, err := sftpClient.Stat("/large_file.bin")
+ require.NoError(t, err)
+ require.Equal(t, int64(size), info.Size())
+
+ // Verify content
+ readFile, err := sftpClient.Open("/large_file.bin")
+ require.NoError(t, err)
+ content, err := io.ReadAll(readFile)
+ require.NoError(t, err)
+ readFile.Close()
+ require.Equal(t, testData, content)
+
+ // Clean up
+ err = sftpClient.Remove("/large_file.bin")
+ require.NoError(t, err)
+ })
+}
+
+// TestStatOperations tests Stat and Lstat operations
+func TestStatOperations(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ sftpClient, sshConn, err := fw.ConnectSFTP("testuser", "testuserpassword")
+ require.NoError(t, err, "failed to connect as testuser")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ // Create a test file
+ testContent := []byte("stat test content")
+ file, err := sftpClient.Create("/stat_test.txt")
+ require.NoError(t, err)
+ _, err = file.Write(testContent)
+ require.NoError(t, err)
+ file.Close()
+
+ t.Run("StatFile", func(t *testing.T) {
+ info, err := sftpClient.Stat("/stat_test.txt")
+ require.NoError(t, err)
+ require.Equal(t, "stat_test.txt", info.Name())
+ require.Equal(t, int64(len(testContent)), info.Size())
+ require.False(t, info.IsDir())
+ })
+
+ t.Run("StatDirectory", func(t *testing.T) {
+ err := sftpClient.Mkdir("/stat_dir")
+ require.NoError(t, err)
+
+ info, err := sftpClient.Stat("/stat_dir")
+ require.NoError(t, err)
+ require.Equal(t, "stat_dir", info.Name())
+ require.True(t, info.IsDir())
+
+ // Clean up
+ err = sftpClient.RemoveDirectory("/stat_dir")
+ require.NoError(t, err)
+ })
+
+ t.Run("StatRoot", func(t *testing.T) {
+ // Should be able to stat "/" which maps to user's home directory
+ info, err := sftpClient.Stat("/")
+ require.NoError(t, err, "should be able to stat root (home) directory")
+ require.True(t, info.IsDir(), "root should be a directory")
+ })
+
+ // Clean up
+ err = sftpClient.Remove("/stat_test.txt")
+ require.NoError(t, err)
+}
+
+// TestWalk tests walking directory trees
+func TestWalk(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ sftpClient, sshConn, err := fw.ConnectSFTP("testuser", "testuserpassword")
+ require.NoError(t, err, "failed to connect as testuser")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ // Create directory structure
+ err = sftpClient.MkdirAll("/walk/a/b")
+ require.NoError(t, err)
+ err = sftpClient.MkdirAll("/walk/c")
+ require.NoError(t, err)
+
+ // Create files
+ for _, p := range []string{"/walk/file1.txt", "/walk/a/file2.txt", "/walk/a/b/file3.txt", "/walk/c/file4.txt"} {
+ file, err := sftpClient.Create(p)
+ require.NoError(t, err)
+ file.Write([]byte("test"))
+ file.Close()
+ }
+
+ t.Run("WalkEntireTree", func(t *testing.T) {
+ var paths []string
+ walker := sftpClient.Walk("/walk")
+ for walker.Step() {
+ if walker.Err() != nil {
+ continue
+ }
+ paths = append(paths, walker.Path())
+ }
+
+ // Should find all directories and files
+ require.Contains(t, paths, "/walk")
+ require.Contains(t, paths, "/walk/a")
+ require.Contains(t, paths, "/walk/a/b")
+ require.Contains(t, paths, "/walk/c")
+ })
+
+ // Clean up
+ for _, p := range []string{"/walk/file1.txt", "/walk/a/file2.txt", "/walk/a/b/file3.txt", "/walk/c/file4.txt"} {
+ require.NoError(t, sftpClient.Remove(p))
+ }
+ for _, p := range []string{"/walk/a/b", "/walk/a", "/walk/c", "/walk"} {
+ require.NoError(t, sftpClient.RemoveDirectory(p))
+ }
+}
+
+// TestCurrentWorkingDirectory tests that Getwd and Chdir work correctly
+func TestCurrentWorkingDirectory(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ sftpClient, sshConn, err := fw.ConnectSFTP("testuser", "testuserpassword")
+ require.NoError(t, err, "failed to connect as testuser")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ // Create test directory
+ err = sftpClient.Mkdir("/cwd_test")
+ require.NoError(t, err)
+
+ t.Run("GetCurrentDir", func(t *testing.T) {
+ cwd, err := sftpClient.Getwd()
+ require.NoError(t, err)
+ // The initial working directory should be the user's home directory
+ // which from the user's perspective is "/"
+ require.Equal(t, "/", cwd, "initial working directory should be the virtual root")
+ })
+
+ t.Run("ChangeAndCreate", func(t *testing.T) {
+ // Create file in subdirectory using relative path after chdir
+ // Note: pkg/sftp doesn't support Chdir, so we test using absolute paths
+ file, err := sftpClient.Create("/cwd_test/relative_file.txt")
+ require.NoError(t, err)
+ file.Write([]byte("test"))
+ file.Close()
+
+ // Verify using absolute path
+ _, err = sftpClient.Stat("/cwd_test/relative_file.txt")
+ require.NoError(t, err)
+
+ // Clean up
+ sftpClient.Remove("/cwd_test/relative_file.txt")
+ })
+
+ // Clean up
+ err = sftpClient.RemoveDirectory("/cwd_test")
+ require.NoError(t, err)
+}
+
+// TestPathEdgeCases tests various edge cases in path handling
+func TestPathEdgeCases(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ sftpClient, sshConn, err := fw.ConnectSFTP("testuser", "testuserpassword")
+ require.NoError(t, err, "failed to connect as testuser")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ t.Run("PathWithDotDot", func(t *testing.T) {
+ // Create directory structure
+ err := sftpClient.MkdirAll("/edge/subdir")
+ require.NoError(t, err)
+
+ // Create file using path with ..
+ file, err := sftpClient.Create("/edge/subdir/../file.txt")
+ require.NoError(t, err)
+ file.Write([]byte("test"))
+ file.Close()
+
+ // Verify file was created in /edge
+ _, err = sftpClient.Stat("/edge/file.txt")
+ require.NoError(t, err, "file should be created in parent directory")
+
+ // Clean up
+ sftpClient.Remove("/edge/file.txt")
+ sftpClient.RemoveDirectory("/edge/subdir")
+ sftpClient.RemoveDirectory("/edge")
+ })
+
+ t.Run("PathWithTrailingSlash", func(t *testing.T) {
+ err := sftpClient.Mkdir("/trailing")
+ require.NoError(t, err)
+
+ // Stat with trailing slash
+ info, err := sftpClient.Stat("/trailing/")
+ require.NoError(t, err)
+ require.True(t, info.IsDir())
+
+ // Clean up
+ sftpClient.RemoveDirectory("/trailing")
+ })
+
+ t.Run("CreateFileAtRootPath", func(t *testing.T) {
+ // This is the exact scenario from issue #7470
+ // User with HomeDir="/sftp/testuser" uploads to "/"
+ file, err := sftpClient.Create("/issue7470.txt")
+ require.NoError(t, err, "should be able to create file at / (issue #7470)")
+ file.Write([]byte("This tests the fix for issue #7470"))
+ file.Close()
+
+ // Verify
+ _, err = sftpClient.Stat("/issue7470.txt")
+ require.NoError(t, err)
+
+ // Clean up
+ sftpClient.Remove("/issue7470.txt")
+ })
+
+ // Security test: path traversal attacks should be blocked
+ t.Run("PathTraversalPrevention", func(t *testing.T) {
+ // User's HomeDir is "/sftp/testuser"
+ // Attempting to escape via "../.." should NOT create files outside home directory
+
+ // First, create a valid file to ensure we can write
+ validFile, err := sftpClient.Create("/valid.txt")
+ require.NoError(t, err)
+ validFile.Write([]byte("valid"))
+ validFile.Close()
+
+ // Try various path traversal attempts
+ // These should either:
+ // 1. Be blocked (error returned), OR
+ // 2. Be safely resolved to stay within home directory
+
+ traversalPaths := []string{
+ "/../escape.txt",
+ "/../../escape.txt",
+ "/../../../escape.txt",
+ "/subdir/../../escape.txt",
+ "/./../../escape.txt",
+ }
+
+ for _, traversalPath := range traversalPaths {
+ t.Run(traversalPath, func(t *testing.T) {
+ // Note: The pkg/sftp client sanitizes paths locally before sending them to the server.
+ // So "/../escape.txt" becomes "/escape.txt" on the wire.
+ // Therefore, we cannot trigger the server-side path traversal block with this client.
+ // Instead, we verify that the file is created successfully within the jail (contained).
+ // The server-side protection logic is verified in unit tests (sftpd/sftp_server_test.go).
+
+ file, err := sftpClient.Create(traversalPath)
+ require.NoError(t, err, "creation should succeed because client sanitizes path")
+ file.Close()
+
+ // Clean up
+ err = sftpClient.Remove(traversalPath)
+ require.NoError(t, err)
+ })
+ }
+
+ // Clean up
+ sftpClient.Remove("/valid.txt")
+ })
+}
+
+// TestFileContent tests reading and writing file content correctly
+func TestFileContent(t *testing.T) {
+ if testing.Short() {
+ t.Skip("skipping integration test in short mode")
+ }
+
+ config := DefaultTestConfig()
+ config.EnableDebug = testing.Verbose()
+
+ fw := NewSftpTestFramework(t, config)
+ err := fw.Setup(config)
+ require.NoError(t, err, "failed to setup test framework")
+ defer fw.Cleanup()
+
+ sftpClient, sshConn, err := fw.ConnectSFTP("testuser", "testuserpassword")
+ require.NoError(t, err, "failed to connect as testuser")
+ defer sshConn.Close()
+ defer sftpClient.Close()
+
+ t.Run("BinaryContent", func(t *testing.T) {
+ // Create binary data with all byte values
+ data := make([]byte, 256)
+ for i := 0; i < 256; i++ {
+ data[i] = byte(i)
+ }
+
+ file, err := sftpClient.Create("/binary.bin")
+ require.NoError(t, err)
+ n, err := file.Write(data)
+ require.NoError(t, err)
+ require.Equal(t, 256, n)
+ file.Close()
+
+ // Read back
+ readFile, err := sftpClient.Open("/binary.bin")
+ require.NoError(t, err)
+ content, err := io.ReadAll(readFile)
+ require.NoError(t, err)
+ readFile.Close()
+
+ require.Equal(t, data, content, "binary content should match")
+
+ // Clean up
+ sftpClient.Remove("/binary.bin")
+ })
+
+ t.Run("EmptyFile", func(t *testing.T) {
+ file, err := sftpClient.Create("/empty.txt")
+ require.NoError(t, err)
+ file.Close()
+
+ info, err := sftpClient.Stat("/empty.txt")
+ require.NoError(t, err)
+ require.Equal(t, int64(0), info.Size())
+
+ // Clean up
+ sftpClient.Remove("/empty.txt")
+ })
+
+ t.Run("UnicodeFilename", func(t *testing.T) {
+ filename := "/文仢名.txt"
+ content := []byte("Unicode content: δ½ ε₯½δΈ–η•Œ")
+
+ file, err := sftpClient.Create(filename)
+ require.NoError(t, err)
+ file.Write(content)
+ file.Close()
+
+ // Read back
+ readFile, err := sftpClient.Open(filename)
+ require.NoError(t, err)
+ readContent, err := io.ReadAll(readFile)
+ require.NoError(t, err)
+ readFile.Close()
+
+ require.Equal(t, content, readContent)
+
+ // Verify in listing
+ files, err := sftpClient.ReadDir("/")
+ require.NoError(t, err)
+ found := false
+ for _, f := range files {
+ if f.Name() == path.Base(filename) {
+ found = true
+ break
+ }
+ }
+ require.True(t, found, "should find unicode filename in listing")
+
+ // Clean up
+ sftpClient.Remove(filename)
+ })
+}
+
diff --git a/test/sftp/framework.go b/test/sftp/framework.go
new file mode 100644
index 000000000..5572eac28
--- /dev/null
+++ b/test/sftp/framework.go
@@ -0,0 +1,423 @@
+package sftp
+
+import (
+ "fmt"
+ "net"
+ "os"
+ "os/exec"
+ "path/filepath"
+ "runtime"
+ "syscall"
+ "testing"
+ "time"
+
+ "github.com/pkg/sftp"
+ "github.com/stretchr/testify/require"
+ "golang.org/x/crypto/ssh"
+)
+
+// SftpTestFramework provides utilities for SFTP integration testing
+type SftpTestFramework struct {
+ t *testing.T
+ tempDir string
+ dataDir string
+ masterProcess *os.Process
+ volumeProcess *os.Process
+ filerProcess *os.Process
+ sftpProcess *os.Process
+ masterAddr string
+ volumeAddr string
+ filerAddr string
+ sftpAddr string
+ weedBinary string
+ userStoreFile string
+ hostKeyFile string
+ isSetup bool
+ skipCleanup bool
+}
+
+// TestConfig holds configuration for SFTP tests
+type TestConfig struct {
+ NumVolumes int
+ EnableDebug bool
+ SkipCleanup bool // for debugging failed tests
+ UserStoreFile string
+}
+
+// DefaultTestConfig returns a default configuration for SFTP tests
+func DefaultTestConfig() *TestConfig {
+ return &TestConfig{
+ NumVolumes: 3,
+ EnableDebug: false,
+ SkipCleanup: false,
+ UserStoreFile: "",
+ }
+}
+
+// NewSftpTestFramework creates a new SFTP testing framework
+func NewSftpTestFramework(t *testing.T, config *TestConfig) *SftpTestFramework {
+ if config == nil {
+ config = DefaultTestConfig()
+ }
+
+ tempDir, err := os.MkdirTemp("", "seaweedfs_sftp_test_")
+ require.NoError(t, err)
+
+ // Generate SSH host key for SFTP server
+ hostKeyFile := filepath.Join(tempDir, "ssh_host_key")
+ cmd := exec.Command("ssh-keygen", "-t", "ed25519", "-f", hostKeyFile, "-N", "")
+ err = cmd.Run()
+ require.NoError(t, err, "failed to generate SSH host key")
+
+ // Use provided userstore or copy the test one
+ userStoreFile := config.UserStoreFile
+ if userStoreFile == "" {
+ // Copy test userstore to temp dir
+ userStoreFile = filepath.Join(tempDir, "userstore.json")
+ testDataPath := findTestDataPath()
+ input, err := os.ReadFile(filepath.Join(testDataPath, "userstore.json"))
+ require.NoError(t, err, "failed to read test userstore.json")
+ err = os.WriteFile(userStoreFile, input, 0644)
+ require.NoError(t, err, "failed to write userstore.json")
+ }
+
+ return &SftpTestFramework{
+ t: t,
+ tempDir: tempDir,
+ dataDir: filepath.Join(tempDir, "data"),
+ masterAddr: "127.0.0.1:19333",
+ volumeAddr: "127.0.0.1:18080",
+ filerAddr: "127.0.0.1:18888",
+ sftpAddr: "127.0.0.1:12022",
+ weedBinary: findWeedBinary(),
+ userStoreFile: userStoreFile,
+ hostKeyFile: hostKeyFile,
+ isSetup: false,
+ }
+}
+
+// Setup starts SeaweedFS cluster with SFTP server
+func (f *SftpTestFramework) Setup(config *TestConfig) error {
+ if f.isSetup {
+ return fmt.Errorf("framework already setup")
+ }
+
+ // Create all data directories
+ dirs := []string{
+ f.dataDir,
+ filepath.Join(f.dataDir, "master"),
+ filepath.Join(f.dataDir, "volume"),
+ }
+ for _, dir := range dirs {
+ if err := os.MkdirAll(dir, 0755); err != nil {
+ return fmt.Errorf("failed to create directory %s: %v", dir, err)
+ }
+ }
+
+ // Start master
+ if err := f.startMaster(config); err != nil {
+ return fmt.Errorf("failed to start master: %v", err)
+ }
+
+ // Wait for master to be ready
+ if err := f.waitForService(f.masterAddr, 30*time.Second); err != nil {
+ return fmt.Errorf("master not ready: %v", err)
+ }
+
+ // Start volume server
+ if err := f.startVolumeServer(config); err != nil {
+ return fmt.Errorf("failed to start volume server: %v", err)
+ }
+
+ // Wait for volume server to be ready
+ if err := f.waitForService(f.volumeAddr, 30*time.Second); err != nil {
+ return fmt.Errorf("volume server not ready: %v", err)
+ }
+
+ // Start filer
+ if err := f.startFiler(config); err != nil {
+ return fmt.Errorf("failed to start filer: %v", err)
+ }
+
+ // Wait for filer to be ready
+ if err := f.waitForService(f.filerAddr, 30*time.Second); err != nil {
+ return fmt.Errorf("filer not ready: %v", err)
+ }
+
+ // Start SFTP server
+ if err := f.startSftpServer(config); err != nil {
+ return fmt.Errorf("failed to start SFTP server: %v", err)
+ }
+
+ // Wait for SFTP server to be ready
+ if err := f.waitForService(f.sftpAddr, 30*time.Second); err != nil {
+ return fmt.Errorf("SFTP server not ready: %v", err)
+ }
+
+ // Additional wait for all services to stabilize (gRPC endpoints)
+ time.Sleep(500 * time.Millisecond)
+
+ f.skipCleanup = config.SkipCleanup
+ f.isSetup = true
+ return nil
+}
+
+// Cleanup stops all processes and removes temporary files
+func (f *SftpTestFramework) Cleanup() {
+ // Stop processes in reverse order
+ processes := []*os.Process{f.sftpProcess, f.filerProcess, f.volumeProcess, f.masterProcess}
+ for _, proc := range processes {
+ if proc != nil {
+ proc.Signal(syscall.SIGTERM)
+ proc.Wait()
+ }
+ }
+
+ // Remove temp directory
+ if !f.skipCleanup {
+ os.RemoveAll(f.tempDir)
+ }
+}
+
+// GetSftpAddr returns the SFTP server address
+func (f *SftpTestFramework) GetSftpAddr() string {
+ return f.sftpAddr
+}
+
+// GetFilerAddr returns the filer address
+func (f *SftpTestFramework) GetFilerAddr() string {
+ return f.filerAddr
+}
+
+// ConnectSFTP creates an SFTP client connection with the given credentials
+func (f *SftpTestFramework) ConnectSFTP(username, password string) (*sftp.Client, *ssh.Client, error) {
+ // Load the known host public key for verification
+ hostKeyCallback, err := f.getHostKeyCallback()
+ if err != nil {
+ return nil, nil, fmt.Errorf("failed to get host key callback: %v", err)
+ }
+
+ config := &ssh.ClientConfig{
+ User: username,
+ Auth: []ssh.AuthMethod{
+ ssh.Password(password),
+ },
+ HostKeyCallback: hostKeyCallback,
+ Timeout: 5 * time.Second,
+ }
+
+ sshConn, err := ssh.Dial("tcp", f.sftpAddr, config)
+ if err != nil {
+ return nil, nil, fmt.Errorf("failed to connect SSH: %v", err)
+ }
+
+ sftpClient, err := sftp.NewClient(sshConn)
+ if err != nil {
+ sshConn.Close()
+ return nil, nil, fmt.Errorf("failed to create SFTP client: %v", err)
+ }
+
+ return sftpClient, sshConn, nil
+}
+
+// getHostKeyCallback returns a callback that verifies the server's host key
+// matches the known test server key we generated
+func (f *SftpTestFramework) getHostKeyCallback() (ssh.HostKeyCallback, error) {
+ // Read the public key file generated alongside the private key
+ pubKeyFile := f.hostKeyFile + ".pub"
+ pubKeyBytes, err := os.ReadFile(pubKeyFile)
+ if err != nil {
+ return nil, fmt.Errorf("failed to read host public key: %v", err)
+ }
+
+ // Parse the public key
+ pubKey, _, _, _, err := ssh.ParseAuthorizedKey(pubKeyBytes)
+ if err != nil {
+ return nil, fmt.Errorf("failed to parse host public key: %v", err)
+ }
+
+ // Return a callback that verifies the server key matches our known key
+ return ssh.FixedHostKey(pubKey), nil
+}
+
+// startMaster starts the SeaweedFS master server
+func (f *SftpTestFramework) startMaster(config *TestConfig) error {
+ args := []string{
+ "master",
+ "-ip=127.0.0.1",
+ "-port=19333",
+ "-mdir=" + filepath.Join(f.dataDir, "master"),
+ "-raftBootstrap",
+ "-peers=none",
+ }
+
+ cmd := exec.Command(f.weedBinary, args...)
+ cmd.Dir = f.tempDir
+ if config.EnableDebug {
+ cmd.Stdout = os.Stdout
+ cmd.Stderr = os.Stderr
+ }
+ if err := cmd.Start(); err != nil {
+ return err
+ }
+ f.masterProcess = cmd.Process
+ return nil
+}
+
+// startVolumeServer starts SeaweedFS volume server
+func (f *SftpTestFramework) startVolumeServer(config *TestConfig) error {
+ args := []string{
+ "volume",
+ "-mserver=" + f.masterAddr,
+ "-ip=127.0.0.1",
+ "-port=18080",
+ "-dir=" + filepath.Join(f.dataDir, "volume"),
+ fmt.Sprintf("-max=%d", config.NumVolumes),
+ }
+
+ cmd := exec.Command(f.weedBinary, args...)
+ cmd.Dir = f.tempDir
+ if config.EnableDebug {
+ cmd.Stdout = os.Stdout
+ cmd.Stderr = os.Stderr
+ }
+ if err := cmd.Start(); err != nil {
+ return err
+ }
+ f.volumeProcess = cmd.Process
+ return nil
+}
+
+// startFiler starts the SeaweedFS filer server
+func (f *SftpTestFramework) startFiler(config *TestConfig) error {
+ args := []string{
+ "filer",
+ "-master=" + f.masterAddr,
+ "-ip=127.0.0.1",
+ "-port=18888",
+ }
+
+ cmd := exec.Command(f.weedBinary, args...)
+ cmd.Dir = f.tempDir
+ if config.EnableDebug {
+ cmd.Stdout = os.Stdout
+ cmd.Stderr = os.Stderr
+ }
+ if err := cmd.Start(); err != nil {
+ return err
+ }
+ f.filerProcess = cmd.Process
+ return nil
+}
+
+// startSftpServer starts the SeaweedFS SFTP server
+func (f *SftpTestFramework) startSftpServer(config *TestConfig) error {
+ args := []string{
+ "sftp",
+ "-filer=" + f.filerAddr,
+ "-ip.bind=127.0.0.1",
+ "-port=12022",
+ "-sshPrivateKey=" + f.hostKeyFile,
+ "-userStoreFile=" + f.userStoreFile,
+ }
+
+ cmd := exec.Command(f.weedBinary, args...)
+ cmd.Dir = f.tempDir
+ if config.EnableDebug {
+ cmd.Stdout = os.Stdout
+ cmd.Stderr = os.Stderr
+ }
+ if err := cmd.Start(); err != nil {
+ return err
+ }
+ f.sftpProcess = cmd.Process
+ return nil
+}
+
+// waitForService waits for a service to be available
+func (f *SftpTestFramework) waitForService(addr string, timeout time.Duration) error {
+ deadline := time.Now().Add(timeout)
+ for time.Now().Before(deadline) {
+ conn, err := net.DialTimeout("tcp", addr, 1*time.Second)
+ if err == nil {
+ conn.Close()
+ return nil
+ }
+ time.Sleep(100 * time.Millisecond)
+ }
+ return fmt.Errorf("service at %s not ready within timeout", addr)
+}
+
+// findWeedBinary locates the weed binary
+// Prefers local build over system-installed weed to ensure we test the latest code
+func findWeedBinary() string {
+ // Get the directory where this source file is located
+ // This ensures we find the locally built weed binary first
+ _, thisFile, _, ok := runtime.Caller(0)
+ if ok {
+ thisDir := filepath.Dir(thisFile)
+ // From test/sftp/, the weed binary should be at ../../weed/weed
+ candidates := []string{
+ filepath.Join(thisDir, "../../weed/weed"),
+ filepath.Join(thisDir, "../weed/weed"),
+ }
+ for _, candidate := range candidates {
+ if _, err := os.Stat(candidate); err == nil {
+ abs, _ := filepath.Abs(candidate)
+ return abs
+ }
+ }
+ }
+
+ // Try relative paths from current working directory
+ cwd, _ := os.Getwd()
+ candidates := []string{
+ filepath.Join(cwd, "../../weed/weed"),
+ filepath.Join(cwd, "../weed/weed"),
+ filepath.Join(cwd, "./weed"),
+ }
+ for _, candidate := range candidates {
+ if _, err := os.Stat(candidate); err == nil {
+ abs, _ := filepath.Abs(candidate)
+ return abs
+ }
+ }
+
+ // Fallback to PATH only if local build not found
+ if path, err := exec.LookPath("weed"); err == nil {
+ return path
+ }
+
+ // Default fallback
+ return "weed"
+}
+
+// findTestDataPath locates the testdata directory
+func findTestDataPath() string {
+ // Get the directory where this source file is located
+ _, thisFile, _, ok := runtime.Caller(0)
+ if ok {
+ thisDir := filepath.Dir(thisFile)
+ testDataPath := filepath.Join(thisDir, "testdata")
+ if _, err := os.Stat(testDataPath); err == nil {
+ return testDataPath
+ }
+ }
+
+ // Try relative paths from current working directory
+ cwd, _ := os.Getwd()
+ candidates := []string{
+ filepath.Join(cwd, "testdata"),
+ filepath.Join(cwd, "../sftp/testdata"),
+ filepath.Join(cwd, "test/sftp/testdata"),
+ }
+
+ for _, candidate := range candidates {
+ if _, err := os.Stat(candidate); err == nil {
+ return candidate
+ }
+ }
+
+ return "./testdata"
+}
+
diff --git a/test/sftp/go.mod b/test/sftp/go.mod
new file mode 100644
index 000000000..34d9053a8
--- /dev/null
+++ b/test/sftp/go.mod
@@ -0,0 +1,17 @@
+module seaweedfs-sftp-tests
+
+go 1.24.0
+
+require (
+ github.com/pkg/sftp v1.13.7
+ github.com/stretchr/testify v1.10.0
+ golang.org/x/crypto v0.45.0
+)
+
+require (
+ github.com/davecgh/go-spew v1.1.1 // indirect
+ github.com/kr/fs v0.1.0 // indirect
+ github.com/pmezard/go-difflib v1.0.0 // indirect
+ golang.org/x/sys v0.38.0 // indirect
+ gopkg.in/yaml.v3 v3.0.1 // indirect
+)
diff --git a/test/sftp/go.sum b/test/sftp/go.sum
new file mode 100644
index 000000000..112e6f88a
--- /dev/null
+++ b/test/sftp/go.sum
@@ -0,0 +1,64 @@
+github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
+github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/kr/fs v0.1.0 h1:Jskdu9ieNAYnjxsi0LbQp1ulIKZV1LAFgK1tWhpZgl8=
+github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg=
+github.com/pkg/sftp v1.13.7 h1:uv+I3nNJvlKZIQGSr8JVQLNHFU9YhhNpvC14Y6KgmSM=
+github.com/pkg/sftp v1.13.7/go.mod h1:KMKI0t3T6hfA+lTR/ssZdunHo+uwq7ghoN09/FSu3DY=
+github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
+github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
+github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
+github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA=
+github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY=
+github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY=
+golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
+golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
+golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4=
+golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q=
+golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4=
+golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4=
+golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs=
+golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
+golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
+golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs=
+golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg=
+golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/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-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA=
+golang.org/x/sys v0.38.0 h1:3yZWxaJjBmCWXqhN1qh02AkOnCQ1poK6oF+a7xWL6Gc=
+golang.org/x/sys v0.38.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks=
+golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
+golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
+golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k=
+golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo=
+golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0=
+golang.org/x/term v0.37.0 h1:8EGAD0qCmHYZg6J17DvsMy9/wJ7/D/4pV/wfnld5lTU=
+golang.org/x/term v0.37.0/go.mod h1:5pB4lxRNYYVZuTLmy8oR2BH8dflOR+IbTYFD8fi3254=
+golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
+golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ=
+golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
+golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8=
+golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU=
+golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc=
+golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU=
+golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
+gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
+gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
diff --git a/test/sftp/testdata/userstore.json b/test/sftp/testdata/userstore.json
new file mode 100644
index 000000000..66d78dd1d
--- /dev/null
+++ b/test/sftp/testdata/userstore.json
@@ -0,0 +1,37 @@
+[
+ {
+ "Username": "admin",
+ "Password": "adminpassword",
+ "PublicKeys": [],
+ "HomeDir": "/",
+ "Permissions": {
+ "/": ["*"]
+ },
+ "Uid": 0,
+ "Gid": 0
+ },
+ {
+ "Username": "testuser",
+ "Password": "testuserpassword",
+ "PublicKeys": [],
+ "HomeDir": "/sftp/testuser",
+ "Permissions": {
+ "/sftp/testuser": ["*"]
+ },
+ "Uid": 1001,
+ "Gid": 1001
+ },
+ {
+ "Username": "readonly",
+ "Password": "readonlypassword",
+ "PublicKeys": [],
+ "HomeDir": "/public",
+ "Permissions": {
+ "/public": ["read", "list"]
+ },
+ "Uid": 1002,
+ "Gid": 1002
+ }
+]
+
+