Files
seaweedFS/weed/s3api/s3api_object_handlers_list.go
Chris Lu 403592bb9f Add Spark Iceberg catalog integration tests and CI support (#8242)
* Add Spark Iceberg catalog integration tests and CI support

Implement comprehensive integration tests for Spark with SeaweedFS Iceberg REST catalog:
- Basic CRUD operations (Create, Read, Update, Delete) on Iceberg tables
- Namespace (database) management
- Data insertion, querying, and deletion
- Time travel capabilities via snapshot versioning
- Compatible with SeaweedFS S3 and Iceberg REST endpoints

Tests mirror the structure of existing Trino integration tests but use Spark's
Python SQL API and PySpark for testing.

Add GitHub Actions CI job for spark-iceberg-catalog-tests in s3-tables-tests.yml
to automatically run Spark integration tests on pull requests.

* fmt

* Fix Spark integration tests - code review feedback

* go mod tidy

* Add go mod tidy step to integration test jobs

Add 'go mod tidy' step before test runs for all integration test jobs:
- s3-tables-tests
- iceberg-catalog-tests
- trino-iceberg-catalog-tests
- spark-iceberg-catalog-tests

This ensures dependencies are clean before running tests.

* Fix remaining Spark operations test issues

Address final code review comments:

Setup & Initialization:
- Add waitForSparkReady() helper function that polls Spark readiness
  with backoff instead of hardcoded 10-second sleep
- Extract setupSparkTestEnv() helper to reduce boilerplate duplication
  between TestSparkCatalogBasicOperations and TestSparkTimeTravel
- Both tests now use helpers for consistent, reliable setup

Assertions & Validation:
- Make setup-critical operations (namespace, table creation, initial
  insert) use t.Fatalf instead of t.Errorf to fail fast
- Validate setupSQL output in TestSparkTimeTravel and fail if not
  'Setup complete'
- Add validation after second INSERT in TestSparkTimeTravel:
  verify row count increased to 2 before time travel test
- Add context to error messages with namespace and tableName params

Code Quality:
- Remove code duplication between test functions
- All critical paths now properly validated
- Consistent error handling throughout

* Fix go vet errors in S3 Tables tests

Fixes:
1. setup_test.go (Spark):
   - Add missing import: github.com/testcontainers/testcontainers-go/wait
   - Use wait.ForLog instead of undefined testcontainers.NewLogStrategy
   - Remove unused strings import

2. trino_catalog_test.go:
   - Use net.JoinHostPort instead of fmt.Sprintf for address formatting
   - Properly handles IPv6 addresses by wrapping them in brackets

* Use weed mini for simpler SeaweedFS startup

Replace complex multi-process startup (master, volume, filer, s3)
with single 'weed mini' command that starts all services together.

Benefits:
- Simpler, more reliable startup
- Single weed mini process vs 4 separate processes
- Automatic coordination between components
- Better port management with no manual coordination

Changes:
- Remove separate master, volume, filer process startup
- Use weed mini with -master.port, -filer.port, -s3.port flags
- Keep Iceberg REST as separate service (still needed)
- Increase timeout to 15s for port readiness (weed mini startup)
- Remove volumePort and filerProcess fields from TestEnvironment
- Simplify cleanup to only handle two processes (mini, iceberg rest)

* Clean up dead code and temp directory leaks

Fixes:

1. Remove dead s3Process field and cleanup:
   - weed mini bundles S3 gateway, no separate process needed
   - Removed s3Process field from TestEnvironment
   - Removed unnecessary s3Process cleanup code

2. Fix temp config directory leak:
   - Add sparkConfigDir field to TestEnvironment
   - Store returned configDir in writeSparkConfig
   - Clean up sparkConfigDir in Cleanup() with os.RemoveAll
   - Prevents accumulation of temp directories in test runs

3. Simplify Cleanup:
   - Now handles only necessary processes (weed mini, iceberg rest)
   - Removes both seaweedfsDataDir and sparkConfigDir
   - Cleaner shutdown sequence

* Use weed mini's built-in Iceberg REST and fix python binary

Changes:
- Add -s3.port.iceberg flag to weed mini for built-in Iceberg REST Catalog
- Remove separate 'weed server' process for Iceberg REST
- Remove icebergRestProcess field from TestEnvironment
- Simplify Cleanup() to only manage weed mini + Spark
- Add port readiness check for iceberg REST from weed mini
- Set Spark container Cmd to '/bin/sh -c sleep 3600' to keep it running
- Change python to python3 in container.Exec calls

This simplifies to truly one all-in-one weed mini process (master, filer, s3,
iceberg-rest) plus just the Spark container.

* go fmt

* clean up

* bind on a non-loopback IP for container access, aligned Iceberg metadata saves/locations with table locations, and reworked Spark time travel to use TIMESTAMP AS OF   with safe timestamp extraction.

* shared mini start

* Fixed internal directory creation under /buckets so .objects paths can auto-create without failing bucket-name validation, which restores table bucket object writes

* fix path

  Updated table bucket objects to write under `/buckets/<bucket>` and saved Iceberg metadata there, adjusting Spark time-travel timestamp to committed_at +1s. Rebuilt the weed binary (`go
  install ./weed`) and confirmed passing tests for Spark and Trino with focused test commands.

* Updated table bucket creation to stop creating /buckets/.objects and switched Trino REST warehouse to s3://<bucket> to match Iceberg layout.

* Stabilize S3Tables integration tests

* Fix timestamp extraction and remove dead code in bucketDir

* Use table bucket as warehouse in s3tables tests

* Update trino_blog_operations_test.go

* adds the CASCADE option to handle any remaining table metadata/files in the schema directory

* skip namespace not empty
2026-02-08 10:03:53 -08:00

798 lines
27 KiB
Go

package s3api
import (
"context"
"encoding/xml"
"errors"
"fmt"
"io"
"net/http"
"net/url"
"sort"
"strconv"
"strings"
"github.com/aws/aws-sdk-go/service/s3"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
"github.com/seaweedfs/seaweedfs/weed/s3api/s3err"
)
type OptionalString struct {
string
set bool
}
func (o OptionalString) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
if !o.set {
return nil
}
return e.EncodeElement(o.string, startElement)
}
type ListBucketResultV2 struct {
XMLName xml.Name `xml:"http://s3.amazonaws.com/doc/2006-03-01/ ListBucketResult"`
Name string `xml:"Name"`
Prefix string `xml:"Prefix"`
MaxKeys uint16 `xml:"MaxKeys"`
Delimiter string `xml:"Delimiter,omitempty"`
IsTruncated bool `xml:"IsTruncated"`
Contents []ListEntry `xml:"Contents,omitempty"`
CommonPrefixes []PrefixEntry `xml:"CommonPrefixes,omitempty"`
ContinuationToken OptionalString `xml:"ContinuationToken,omitempty"`
NextContinuationToken string `xml:"NextContinuationToken,omitempty"`
EncodingType string `xml:"EncodingType,omitempty"`
KeyCount int `xml:"KeyCount"`
StartAfter string `xml:"StartAfter,omitempty"`
}
func (s3a *S3ApiServer) ListObjectsV2Handler(w http.ResponseWriter, r *http.Request) {
// https://docs.aws.amazon.com/AmazonS3/latest/API/v2-RESTBucketGET.html
// collect parameters
bucket, _ := s3_constants.GetBucketAndObject(r)
originalPrefix, startAfter, delimiter, continuationToken, encodingTypeUrl, fetchOwner, maxKeys, allowUnordered, errCode := getListObjectsV2Args(r.URL.Query())
glog.V(2).Infof("ListObjectsV2Handler bucket=%s prefix=%s marker=%s", bucket, originalPrefix, continuationToken.string)
if errCode != s3err.ErrNone {
s3err.WriteErrorResponse(w, r, errCode)
return
}
// maxKeys is uint16 here; negative values are rejected during parsing.
// AWS S3 compatibility: allow-unordered cannot be used with delimiter
if allowUnordered && delimiter != "" {
s3err.WriteErrorResponse(w, r, s3err.ErrInvalidUnorderedWithDelimiter)
return
}
marker := continuationToken.string
if !continuationToken.set {
marker = startAfter
}
// Adjust marker if it ends with delimiter to skip all entries with that prefix
marker = adjustMarkerForDelimiter(marker, delimiter)
response, err := s3a.listFilerEntries(bucket, originalPrefix, maxKeys, marker, delimiter, encodingTypeUrl, fetchOwner)
if err != nil {
s3err.WriteErrorResponse(w, r, s3err.ErrInternalError)
return
}
if len(response.Contents) == 0 {
if exists, existErr := s3a.bucketExists(bucket); existErr == nil && !exists {
s3err.WriteErrorResponse(w, r, s3err.ErrNoSuchBucket)
return
}
}
responseV2 := &ListBucketResultV2{
Name: response.Name,
CommonPrefixes: response.CommonPrefixes,
Contents: response.Contents,
ContinuationToken: continuationToken,
Delimiter: response.Delimiter,
IsTruncated: response.IsTruncated,
KeyCount: len(response.Contents) + len(response.CommonPrefixes),
MaxKeys: uint16(response.MaxKeys),
NextContinuationToken: response.NextMarker,
Prefix: response.Prefix,
StartAfter: startAfter,
}
if encodingTypeUrl {
responseV2.EncodingType = s3.EncodingTypeUrl
}
glog.V(3).Infof("ListObjectsV2Handler response: %+v", responseV2)
writeSuccessResponseXML(w, r, responseV2)
}
func (s3a *S3ApiServer) ListObjectsV1Handler(w http.ResponseWriter, r *http.Request) {
// https://docs.aws.amazon.com/AmazonS3/latest/API/API_ListObjects.html
// collect parameters
bucket, _ := s3_constants.GetBucketAndObject(r)
originalPrefix, marker, delimiter, encodingTypeUrl, maxKeys, allowUnordered, errCode := getListObjectsV1Args(r.URL.Query())
glog.V(2).Infof("ListObjectsV1Handler bucket=%s prefix=%s marker=%s delimiter=%s maxKeys=%d", bucket, originalPrefix, marker, delimiter, maxKeys)
if errCode != s3err.ErrNone {
s3err.WriteErrorResponse(w, r, errCode)
return
}
if maxKeys < 0 {
s3err.WriteErrorResponse(w, r, s3err.ErrInvalidMaxKeys)
return
}
// AWS S3 compatibility: allow-unordered cannot be used with delimiter
if allowUnordered && delimiter != "" {
s3err.WriteErrorResponse(w, r, s3err.ErrInvalidUnorderedWithDelimiter)
return
}
// Adjust marker if it ends with delimiter to skip all entries with that prefix
marker = adjustMarkerForDelimiter(marker, delimiter)
response, err := s3a.listFilerEntries(bucket, originalPrefix, uint16(maxKeys), marker, delimiter, encodingTypeUrl, true)
if err != nil {
s3err.WriteErrorResponse(w, r, s3err.ErrInternalError)
return
}
if len(response.Contents) == 0 {
if exists, existErr := s3a.bucketExists(bucket); existErr == nil && !exists {
s3err.WriteErrorResponse(w, r, s3err.ErrNoSuchBucket)
return
}
}
glog.V(3).Infof("ListObjectsV1Handler response: %+v", response)
writeSuccessResponseXML(w, r, response)
}
func (s3a *S3ApiServer) listFilerEntries(bucket string, originalPrefix string, maxKeys uint16, originalMarker string, delimiter string, encodingTypeUrl bool, fetchOwner bool) (response ListBucketResult, err error) {
// convert full path prefix into directory name and prefix for entry name
requestDir, prefix, marker := normalizePrefixMarker(originalPrefix, originalMarker)
bucketPrefix := s3a.bucketPrefix(bucket)
reqDir := bucketPrefix[:len(bucketPrefix)-1]
if requestDir != "" {
reqDir = fmt.Sprintf("%s%s", bucketPrefix, requestDir)
}
var contents []ListEntry
var commonPrefixes []PrefixEntry
var doErr error
var nextMarker string
cursor := &ListingCursor{
maxKeys: maxKeys,
prefixEndsOnDelimiter: strings.HasSuffix(originalPrefix, "/") && len(originalMarker) == 0,
}
// Special case: when maxKeys = 0, return empty results immediately with IsTruncated=false
if maxKeys == 0 {
response = ListBucketResult{
Name: bucket,
Prefix: originalPrefix,
Marker: originalMarker,
NextMarker: "",
MaxKeys: int(maxKeys),
Delimiter: delimiter,
IsTruncated: false,
Contents: contents,
CommonPrefixes: commonPrefixes,
}
if encodingTypeUrl {
response.EncodingType = s3.EncodingTypeUrl
}
return
}
// check filer
err = s3a.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
var lastEntryWasCommonPrefix bool
var lastCommonPrefixName string
// Hoist versioning check out of per-entry callback
versioningState, _ := s3a.getVersioningState(bucket)
versioningEnabled := versioningState == "Enabled"
// Helper function to handle dedup/append logic
appendOrDedup := func(newEntry ListEntry) {
if versioningEnabled {
// For versioned buckets, we need to handle duplicates between the main file and the .versions directory
if len(contents) > 0 && contents[len(contents)-1].Key == newEntry.Key {
glog.V(3).Infof("listFilerEntries deduplicating versioned entry: %s", newEntry.Key)
contents[len(contents)-1] = newEntry
} else {
contents = append(contents, newEntry)
cursor.maxKeys--
}
} else {
contents = append(contents, newEntry)
cursor.maxKeys--
}
}
for {
empty := true
nextMarker, doErr = s3a.doListFilerEntries(client, reqDir, prefix, cursor, marker, delimiter, false, bucket, func(dir string, entry *filer_pb.Entry) {
empty = false
dirName, entryName, _ := entryUrlEncode(dir, entry.Name, encodingTypeUrl)
if entry.IsDirectory {
if originalPrefix != "" {
normalizedPrefix := strings.TrimPrefix(strings.TrimSuffix(originalPrefix, "/"), "/")
if normalizedPrefix != "" {
relativePath := strings.TrimPrefix(fmt.Sprintf("%s/%s", dir, entry.Name), bucketPrefix)
relativePath = strings.TrimPrefix(relativePath, "/")
if normalizedPrefix == relativePath && !s3a.hasChildren(bucket, relativePath) && !entry.IsDirectoryKeyObject() {
return
}
}
}
// When delimiter is specified, apply delimiter logic to directory key objects too
if delimiter != "" && entry.IsDirectoryKeyObject() {
// Apply the same delimiter logic as for regular files
var delimiterFound bool
// Use raw dir and entry.Name (not encoded) to ensure consistent handling
// Encoding will be applied after sorting if encodingTypeUrl is set
undelimitedPath := fmt.Sprintf("%s/%s/", dir, entry.Name)[len(bucketPrefix):]
// take into account a prefix if supplied while delimiting.
undelimitedPath = strings.TrimPrefix(undelimitedPath, originalPrefix)
delimitedPath := strings.SplitN(undelimitedPath, delimiter, 2)
if len(delimitedPath) == 2 {
// S3 clients expect the delimited prefix to contain the delimiter and prefix.
delimitedPrefix := originalPrefix + delimitedPath[0] + delimiter
// Check if this CommonPrefix already exists
if !lastEntryWasCommonPrefix || lastCommonPrefixName != delimitedPath[0] {
// New CommonPrefix found
commonPrefixes = append(commonPrefixes, PrefixEntry{
Prefix: delimitedPrefix,
})
cursor.maxKeys--
delimiterFound = true
lastEntryWasCommonPrefix = true
lastCommonPrefixName = delimitedPath[0]
} else {
// This directory object belongs to an existing CommonPrefix, skip it
delimiterFound = true
}
}
// If no delimiter found in the directory object name, treat it as a regular key
if !delimiterFound {
newEntry := newListEntry(s3a, entry, "", dirName, entryName, bucketPrefix, fetchOwner, true, false)
appendOrDedup(newEntry)
lastEntryWasCommonPrefix = false
}
} else if entry.IsDirectoryKeyObject() {
// No delimiter specified, or delimiter doesn't apply - treat as regular key
newEntry := newListEntry(s3a, entry, "", dirName, entryName, bucketPrefix, fetchOwner, true, false)
appendOrDedup(newEntry)
lastEntryWasCommonPrefix = false
// https://docs.aws.amazon.com/AmazonS3/latest/API/API_ListObjectsV2.html
} else if delimiter != "" { // A response can contain CommonPrefixes only if you specify a delimiter.
// Use raw dir and entry.Name (not encoded) to ensure consistent handling
// Encoding will be applied after sorting if encodingTypeUrl is set
commonPrefixes = append(commonPrefixes, PrefixEntry{
Prefix: fmt.Sprintf("%s/%s/", dir, entry.Name)[len(bucketPrefix):],
})
//All of the keys (up to 1,000) rolled up into a common prefix count as a single return when calculating the number of returns.
cursor.maxKeys--
lastEntryWasCommonPrefix = true
lastCommonPrefixName = entry.Name
}
} else {
var delimiterFound bool
if delimiter != "" {
// keys that contain the same string between the prefix and the first occurrence of the delimiter are grouped together as a commonPrefix.
// extract the string between the prefix and the delimiter and add it to the commonPrefixes if it's unique.
undelimitedPath := fmt.Sprintf("%s/%s", dir, entry.Name)[len(bucketPrefix):]
// take into account a prefix if supplied while delimiting.
undelimitedPath = strings.TrimPrefix(undelimitedPath, originalPrefix)
delimitedPath := strings.SplitN(undelimitedPath, delimiter, 2)
if len(delimitedPath) == 2 {
// S3 clients expect the delimited prefix to contain the delimiter and prefix.
delimitedPrefix := originalPrefix + delimitedPath[0] + delimiter
for i := range commonPrefixes {
if commonPrefixes[i].Prefix == delimitedPrefix {
delimiterFound = true
break
}
}
if !delimiterFound {
commonPrefixes = append(commonPrefixes, PrefixEntry{
Prefix: delimitedPrefix,
})
cursor.maxKeys--
delimiterFound = true
lastEntryWasCommonPrefix = true
lastCommonPrefixName = delimitedPath[0]
} else {
// This object belongs to an existing CommonPrefix, skip it
// but continue processing to maintain correct flow
delimiterFound = true
}
}
}
if !delimiterFound {
glog.V(4).Infof("Adding file to contents: %s", entryName)
newEntry := newListEntry(s3a, entry, "", dirName, entryName, bucketPrefix, fetchOwner, false, false)
appendOrDedup(newEntry)
lastEntryWasCommonPrefix = false
}
}
})
if doErr != nil {
if errors.Is(doErr, filer_pb.ErrNotFound) {
empty = true
nextMarker = ""
break
}
return doErr
}
// Adjust nextMarker for CommonPrefixes to include trailing slash (AWS S3 compliance)
if cursor.isTruncated && lastEntryWasCommonPrefix && lastCommonPrefixName != "" {
// For CommonPrefixes, NextMarker should include the trailing slash
if requestDir != "" {
if prefix != "" {
nextMarker = requestDir + "/" + prefix + "/" + lastCommonPrefixName + "/"
} else {
nextMarker = requestDir + "/" + lastCommonPrefixName + "/"
}
} else {
nextMarker = lastCommonPrefixName + "/"
}
} else if cursor.isTruncated {
if requestDir != "" {
if prefix != "" {
nextMarker = requestDir + "/" + prefix + "/" + nextMarker
} else {
nextMarker = requestDir + "/" + nextMarker
}
}
}
if cursor.isTruncated {
break
} else if empty || strings.HasSuffix(originalPrefix, "/") {
nextMarker = ""
break
} else {
// start next loop
marker = nextMarker
}
}
response = ListBucketResult{
Name: bucket,
Prefix: originalPrefix,
Marker: originalMarker,
NextMarker: nextMarker,
MaxKeys: int(maxKeys),
Delimiter: delimiter,
IsTruncated: cursor.isTruncated,
Contents: contents,
CommonPrefixes: commonPrefixes,
}
// Sort CommonPrefixes to match AWS S3 behavior
// AWS S3 treats the delimiter character specially for sorting common prefixes.
// For example, with delimiter '/', 'foo/' should come before 'foo+1/' even though '+' (ASCII 43) < '/' (ASCII 47).
// This custom comparison ensures correct S3-compatible lexicographical ordering.
sort.Slice(response.CommonPrefixes, func(i, j int) bool {
return compareWithDelimiter(response.CommonPrefixes[i].Prefix, response.CommonPrefixes[j].Prefix, delimiter)
})
// URL-encode CommonPrefixes AFTER sorting (if EncodingType=url)
// This ensures proper sort order (on decoded values) and correct encoding in response
if encodingTypeUrl {
response.EncodingType = s3.EncodingTypeUrl
for i := range response.CommonPrefixes {
response.CommonPrefixes[i].Prefix = urlPathEscape(response.CommonPrefixes[i].Prefix)
}
}
return nil
})
return
}
type ListingCursor struct {
maxKeys uint16
isTruncated bool
prefixEndsOnDelimiter bool
}
// the prefix and marker may be in different directories
// normalizePrefixMarker ensures the prefix and marker both starts from the same directory
func normalizePrefixMarker(prefix, marker string) (alignedDir, alignedPrefix, alignedMarker string) {
// alignedDir should not end with "/"
// alignedDir, alignedPrefix, alignedMarker should only have "/" in middle
if len(marker) == 0 {
prefix = strings.Trim(prefix, "/")
} else {
prefix = strings.TrimLeft(prefix, "/")
}
marker = strings.TrimLeft(marker, "/")
if prefix == "" {
return "", "", marker
}
if marker == "" {
alignedDir, alignedPrefix = toDirAndName(prefix)
return
}
if !strings.HasPrefix(marker, prefix) {
// something wrong
return "", prefix, marker
}
if strings.HasPrefix(marker, prefix+"/") {
alignedDir = prefix
alignedPrefix = ""
alignedMarker = marker[len(alignedDir)+1:]
return
}
alignedDir, alignedPrefix = toDirAndName(prefix)
if alignedDir != "" {
alignedMarker = marker[len(alignedDir)+1:]
} else {
alignedMarker = marker
}
return
}
func toDirAndName(dirAndName string) (dir, name string) {
sepIndex := strings.LastIndex(dirAndName, "/")
if sepIndex >= 0 {
dir, name = dirAndName[0:sepIndex], dirAndName[sepIndex+1:]
} else {
name = dirAndName
}
return
}
func toParentAndDescendants(dirAndName string) (dir, name string) {
sepIndex := strings.Index(dirAndName, "/")
if sepIndex >= 0 {
dir, name = dirAndName[0:sepIndex], dirAndName[sepIndex+1:]
} else {
name = dirAndName
}
return
}
func (s3a *S3ApiServer) doListFilerEntries(client filer_pb.SeaweedFilerClient, dir, prefix string, cursor *ListingCursor, marker, delimiter string, inclusiveStartFrom bool, bucket string, eachEntryFn func(dir string, entry *filer_pb.Entry)) (nextMarker string, err error) {
// invariants
// prefix and marker should be under dir, marker may contain "/"
// maxKeys should be updated for each recursion
// glog.V(4).Infof("doListFilerEntries dir: %s, prefix: %s, marker %s, maxKeys: %d, prefixEndsOnDelimiter: %+v", dir, prefix, marker, cursor.maxKeys, cursor.prefixEndsOnDelimiter)
// When listing at bucket root with delimiter '/', prefix can be "/" after normalization.
// Returning early here would incorrectly hide all top-level entries (folders like "Veeam/").
if cursor.maxKeys <= 0 {
return // Don't set isTruncated here - let caller decide based on whether more entries exist
}
if strings.Contains(marker, "/") {
subDir, subMarker := toParentAndDescendants(marker)
// println("doListFilerEntries dir", dir+"/"+subDir, "subMarker", subMarker)
subNextMarker, subErr := s3a.doListFilerEntries(client, dir+"/"+subDir, "", cursor, subMarker, delimiter, false, bucket, eachEntryFn)
if subErr != nil {
err = subErr
return
}
nextMarker = subDir + "/" + subNextMarker
// finished processing this subdirectory
marker = subDir
}
if cursor.isTruncated {
return
}
// now marker is also a direct child of dir
request := &filer_pb.ListEntriesRequest{
Directory: dir,
Prefix: prefix,
Limit: uint32(cursor.maxKeys + 2), // bucket root directory needs to skip additional s3_constants.MultipartUploadsFolder folder
StartFromFileName: marker,
InclusiveStartFrom: inclusiveStartFrom,
}
if cursor.prefixEndsOnDelimiter {
request.Limit = uint32(1)
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stream, listErr := client.ListEntries(ctx, request)
if listErr != nil {
if errors.Is(listErr, filer_pb.ErrNotFound) {
return
}
err = fmt.Errorf("list entries %+v: %w", request, listErr)
return
}
for {
resp, recvErr := stream.Recv()
if recvErr != nil {
if recvErr == io.EOF {
break
} else {
err = fmt.Errorf("iterating entries %+v: %v", request, recvErr)
return
}
}
entry := resp.Entry
if cursor.maxKeys <= 0 {
cursor.isTruncated = true
break
}
// Set nextMarker only when we have quota to process this entry
nextMarker = entry.Name
if cursor.prefixEndsOnDelimiter {
if entry.Name == prefix && entry.IsDirectory {
if delimiter != "/" {
cursor.prefixEndsOnDelimiter = false
}
} else {
continue
}
}
if entry.IsDirectory {
// glog.V(4).Infof("List Dir Entries %s, file: %s, maxKeys %d", dir, entry.Name, cursor.maxKeys)
if entry.Name == s3_constants.MultipartUploadsFolder { // FIXME no need to apply to all directories. this extra also affects maxKeys
continue
}
// Process .versions directories immediately to create logical versioned object entries
// These directories are never traversed (we continue here), so each is only encountered once
if strings.HasSuffix(entry.Name, s3_constants.VersionsFolder) {
// Extract object name from .versions directory name
baseObjectName := strings.TrimSuffix(entry.Name, s3_constants.VersionsFolder)
// Construct full object path relative to bucket
bucketFullPath := s3a.bucketDir(bucket)
bucketRelativePath := strings.TrimPrefix(dir, bucketFullPath)
bucketRelativePath = strings.TrimPrefix(bucketRelativePath, "/")
var fullObjectPath string
if bucketRelativePath == "" {
fullObjectPath = baseObjectName
} else {
fullObjectPath = bucketRelativePath + "/" + baseObjectName
}
// Use metadata from the already-fetched .versions directory entry
if latestVersionEntry, err := s3a.getLatestVersionEntryFromDirectoryEntry(bucket, fullObjectPath, entry); err == nil {
eachEntryFn(dir, latestVersionEntry)
} else if !errors.Is(err, ErrDeleteMarker) {
// Log unexpected errors (delete markers are expected)
glog.V(2).Infof("Skipping versioned object %s due to error: %v", fullObjectPath, err)
}
continue
}
if delimiter != "/" || cursor.prefixEndsOnDelimiter {
// When delimiter is empty (recursive mode), recurse into directories but don't add them to results
// Only files and versioned objects should appear in results
if cursor.prefixEndsOnDelimiter {
cursor.prefixEndsOnDelimiter = false
if entry.IsDirectoryKeyObject() {
eachEntryFn(dir, entry)
}
}
// Recurse into subdirectory - don't add the directory itself to results
subNextMarker, subErr := s3a.doListFilerEntries(client, dir+"/"+entry.Name, "", cursor, "", delimiter, false, bucket, eachEntryFn)
if subErr != nil {
err = fmt.Errorf("doListFilerEntries2: %w", subErr)
return
}
// println("doListFilerEntries2 dir", dir+"/"+entry.Name, "subNextMarker", subNextMarker)
nextMarker = entry.Name + "/" + subNextMarker
if cursor.isTruncated {
return
}
// println("doListFilerEntries2 nextMarker", nextMarker)
} else {
eachEntryFn(dir, entry)
}
} else {
eachEntryFn(dir, entry)
// glog.V(4).Infof("List File Entries %s, file: %s, maxKeys %d", dir, entry.Name, cursor.maxKeys)
}
if cursor.prefixEndsOnDelimiter {
cursor.prefixEndsOnDelimiter = false
}
}
// Versioned directories are processed above (lines 524-546)
return
}
func getListObjectsV2Args(values url.Values) (prefix, startAfter, delimiter string, token OptionalString, encodingTypeUrl bool, fetchOwner bool, maxkeys uint16, allowUnordered bool, errCode s3err.ErrorCode) {
prefix = values.Get("prefix")
token = OptionalString{set: values.Has("continuation-token"), string: values.Get("continuation-token")}
startAfter = values.Get("start-after")
delimiter = values.Get("delimiter")
encodingTypeUrl = values.Get("encoding-type") == s3.EncodingTypeUrl
if values.Get("max-keys") != "" {
if maxKeys, err := strconv.ParseUint(values.Get("max-keys"), 10, 16); err == nil {
maxkeys = uint16(maxKeys)
} else {
// Invalid max-keys value (non-numeric)
errCode = s3err.ErrInvalidMaxKeys
return
}
} else {
maxkeys = maxObjectListSizeLimit
}
fetchOwner = values.Get("fetch-owner") == "true"
allowUnordered = values.Get("allow-unordered") == "true"
errCode = s3err.ErrNone
return
}
func getListObjectsV1Args(values url.Values) (prefix, marker, delimiter string, encodingTypeUrl bool, maxkeys int16, allowUnordered bool, errCode s3err.ErrorCode) {
prefix = values.Get("prefix")
marker = values.Get("marker")
delimiter = values.Get("delimiter")
encodingTypeUrl = values.Get("encoding-type") == "url"
if values.Get("max-keys") != "" {
if maxKeys, err := strconv.ParseInt(values.Get("max-keys"), 10, 16); err == nil {
maxkeys = int16(maxKeys)
} else {
// Invalid max-keys value (non-numeric)
errCode = s3err.ErrInvalidMaxKeys
return
}
} else {
maxkeys = maxObjectListSizeLimit
}
allowUnordered = values.Get("allow-unordered") == "true"
errCode = s3err.ErrNone
return
}
func (s3a *S3ApiServer) ensureDirectoryAllEmpty(filerClient filer_pb.SeaweedFilerClient, parentDir, name string) (isEmpty bool, err error) {
// println("+ ensureDirectoryAllEmpty", dir, name)
glog.V(4).Infof("+ isEmpty %s/%s", parentDir, name)
defer glog.V(4).Infof("- isEmpty %s/%s %v", parentDir, name, isEmpty)
var fileCounter int
var subDirs []string
currentDir := parentDir + "/" + name
var startFrom string
var isExhausted bool
var foundEntry bool
for fileCounter == 0 && !isExhausted && err == nil {
err = filer_pb.SeaweedList(context.Background(), filerClient, currentDir, "", func(entry *filer_pb.Entry, isLast bool) error {
foundEntry = true
if entry.IsOlderDir() {
subDirs = append(subDirs, entry.Name)
} else {
fileCounter++
}
startFrom = entry.Name
isExhausted = isExhausted || isLast
glog.V(4).Infof(" * %s/%s isLast: %t", currentDir, startFrom, isLast)
return nil
}, startFrom, false, 8)
if !foundEntry {
break
}
}
if err != nil {
return false, err
}
if fileCounter > 0 {
return false, nil
}
for _, subDir := range subDirs {
isSubEmpty, subErr := s3a.ensureDirectoryAllEmpty(filerClient, currentDir, subDir)
if subErr != nil {
return false, subErr
}
if !isSubEmpty {
return false, nil
}
}
glog.V(1).Infof("deleting empty folder %s", currentDir)
if err = doDeleteEntry(filerClient, parentDir, name, true, false); err != nil {
return
}
return true, nil
}
// compareWithDelimiter compares two strings for sorting, treating the delimiter character
// as having lower precedence than other characters to match AWS S3 behavior.
// For example, with delimiter '/', 'foo/' should come before 'foo+1/' even though '+' < '/' in ASCII.
// Note: This function assumes delimiter is a single character. Multi-character delimiters will fall back to standard comparison.
func compareWithDelimiter(a, b, delimiter string) bool {
if delimiter == "" {
return a < b
}
// Multi-character delimiters are not supported by AWS S3 in practice,
// but if encountered, fall back to standard byte-wise comparison
if len(delimiter) != 1 {
return a < b
}
delimByte := delimiter[0]
minLen := len(a)
if len(b) < minLen {
minLen = len(b)
}
// Compare character by character
for i := 0; i < minLen; i++ {
charA := a[i]
charB := b[i]
if charA == charB {
continue
}
// Check if either character is the delimiter
isDelimA := charA == delimByte
isDelimB := charB == delimByte
if isDelimA && !isDelimB {
// Delimiter in 'a' should come first
return true
}
if !isDelimA && isDelimB {
// Delimiter in 'b' should come first
return false
}
// Neither or both are delimiters, use normal comparison
return charA < charB
}
// If we get here, one string is a prefix of the other
return len(a) < len(b)
}
// adjustMarkerForDelimiter handles delimiter-ending markers by incrementing them to skip entries with that prefix.
// For example, when continuation token is "boo/", this returns "boo~" to skip all "boo/*" entries
// but still finds any "bop" or later entries. We add a high ASCII character rather than incrementing
// the last character to avoid skipping potential directory entries.
// This is essential for correct S3 list operations with delimiters and CommonPrefixes.
func adjustMarkerForDelimiter(marker, delimiter string) string {
if delimiter == "" || !strings.HasSuffix(marker, delimiter) {
return marker
}
// Remove the trailing delimiter
// This ensures we skip all entries under the prefix but don't skip
// potential directory entries that start with a similar prefix
prefix := strings.TrimSuffix(marker, delimiter)
if len(prefix) == 0 {
return marker
}
return prefix
}