optimize max-keys=2 listing for spark workloads (#19725)

to return results appropriately for versioned buckets, especially
when underlying prefixes have been deleted
This commit is contained in:
Poorna 2024-05-13 07:57:42 -07:00 committed by GitHub
parent 01bfc78535
commit 7752b03add
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194

View File

@ -1460,8 +1460,7 @@ func maxKeysPlusOne(maxKeys int, addOne bool) int {
return maxKeys
}
func (z *erasureServerPools) listObjectsGeneric(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int, v1 bool) (ListObjectsInfo, error) {
var loi ListObjectsInfo
func (z *erasureServerPools) listObjectsGeneric(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int, v1 bool) (loi ListObjectsInfo, err error) {
opts := listPathOptions{
V1: v1,
Bucket: bucket,
@ -1473,123 +1472,8 @@ func (z *erasureServerPools) listObjectsGeneric(ctx context.Context, bucket, pre
AskDisks: globalAPIConfig.getListQuorum(),
}
opts.setBucketMeta(ctx)
ri := logger.GetReqInfo(ctx)
hadoop := ri != nil && strings.Contains(ri.UserAgent, `Hadoop `) && strings.Contains(ri.UserAgent, "scala/")
matches := func() bool {
if prefix == "" {
return false
}
// List of standard files supported by s3a
// that involves a List() on a directory
// where directory is actually an object on
// namespace.
for _, k := range []string{
"_SUCCESS/",
".parquet/",
".csv/",
".json/",
".avro/",
".orc/",
".txt/",
// Add any other files in future
} {
if strings.HasSuffix(prefix, k) {
return true
}
}
return false
}
if hadoop && matches() && delimiter == SlashSeparator && maxKeys == 2 && marker == "" {
// Optimization for Spark/Hadoop workload where spark sends a garbage
// request of this kind
//
// GET /testbucket/?list-type=2&delimiter=%2F&max-keys=2&prefix=parquet%2F_SUCCESS%2F&fetch-owner=false
//
// Here spark is expecting that the List() return empty instead, so from MinIO's point
// of view if we simply do a GetObjectInfo() on this prefix by treating it as an object
// We save a lot of calls over the network.
//
// This happens repeatedly for all objects that are created concurrently() avoiding this
// as a List() call is an important performance improvement.
//
// Spark based s3a committers are a big enough use-case to have this optimization.
//
// A sample code to see the improvements is as follows, this sample code is
// simply a read on JSON from MinIO and write it back as "parquet".
//
// import org.apache.spark.sql.SparkSession
// import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
// object SparkJSONRead {
// def main(args: Array[String]): Unit = {
// val spark:SparkSession = SparkSession.builder()
// .appName("SparkByExample")
// .master("local[1]").getOrCreate()
//
// spark.sparkContext.setLogLevel("ERROR")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.endpoint", "http://minio-lb:9000")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.path.style.access", "true")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", "minioadmin")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", "minioadmin")
//
// val df = spark.read.json("s3a://testbucket/s3.json")
//
// df.write.parquet("s3a://testbucket/parquet/")
// }
// }
objInfo, err := z.GetObjectInfo(ctx, bucket, path.Dir(prefix), ObjectOptions{NoLock: true})
if err == nil {
if opts.Lifecycle != nil {
evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, opts.Replication.Config, objInfo)
if evt.Action.Delete() {
globalExpiryState.enqueueByDays(objInfo, evt, lcEventSrc_s3ListObjects)
if !evt.Action.DeleteRestored() {
// Skip entry if ILM action was DeleteVersionAction or DeleteAction
return loi, nil
}
}
}
return loi, nil
}
if isErrBucketNotFound(err) {
return loi, err
}
if contextCanceled(ctx) {
return ListObjectsInfo{}, ctx.Err()
}
}
if len(prefix) > 0 && maxKeys == 1 && marker == "" {
// Optimization for certain applications like
// - Cohesity
// - Actifio, Splunk etc.
// which send ListObjects requests where the actual object
// itself is the prefix and max-keys=1 in such scenarios
// we can simply verify locally if such an object exists
// to avoid the need for ListObjects().
objInfo, err := z.GetObjectInfo(ctx, bucket, prefix, ObjectOptions{NoLock: true})
if err == nil {
if opts.Lifecycle != nil {
evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, opts.Replication.Config, objInfo)
if evt.Action.Delete() {
globalExpiryState.enqueueByDays(objInfo, evt, lcEventSrc_s3ListObjects)
if !evt.Action.DeleteRestored() {
// Skip entry if ILM action was DeleteVersionAction or DeleteAction
return loi, nil
}
}
}
loi.Objects = append(loi.Objects, objInfo)
return loi, nil
}
if isErrBucketNotFound(err) {
return ListObjectsInfo{}, err
}
if contextCanceled(ctx) {
return ListObjectsInfo{}, ctx.Err()
}
}
listFn := func(ctx context.Context, opts listPathOptions, limitTo int) (ListObjectsInfo, error) {
var loi ListObjectsInfo
merged, err := z.listPath(ctx, &opts)
if err != nil && err != io.EOF {
if !isErrBucketNotFound(err) {
@ -1597,7 +1481,6 @@ func (z *erasureServerPools) listObjectsGeneric(ctx context.Context, bucket, pre
}
return loi, toObjectErr(err, bucket)
}
merged.forwardPast(opts.Marker)
defer merged.truncate(0) // Release when returning
@ -1608,8 +1491,8 @@ func (z *erasureServerPools) listObjectsGeneric(ctx context.Context, bucket, pre
// Default is recursive, if delimiter is set then list non recursive.
objects := merged.fileInfos(bucket, prefix, delimiter)
loi.IsTruncated = err == nil && len(objects) > 0
if maxKeys > 0 && len(objects) > maxKeys {
objects = objects[:maxKeys]
if limitTo > 0 && len(objects) > limitTo {
objects = objects[:limitTo]
loi.IsTruncated = true
}
for _, obj := range objects {
@ -1650,8 +1533,164 @@ func (z *erasureServerPools) listObjectsGeneric(ctx context.Context, bucket, pre
if loi.NextMarker != "" {
loi.NextMarker = opts.encodeMarker(loi.NextMarker)
}
return loi, nil
}
ri := logger.GetReqInfo(ctx)
hadoop := ri != nil && strings.Contains(ri.UserAgent, `Hadoop `) && strings.Contains(ri.UserAgent, "scala/")
matches := func() bool {
if prefix == "" {
return false
}
// List of standard files supported by s3a
// that involves a List() on a directory
// where directory is actually an object on
// namespace.
for _, k := range []string{
"_SUCCESS/",
".parquet/",
".csv/",
".json/",
".avro/",
".orc/",
".txt/",
// Add any other files in future
} {
if strings.HasSuffix(prefix, k) {
return true
}
}
return false
}
if hadoop && delimiter == SlashSeparator && maxKeys == 2 && marker == "" {
// Optimization for Spark/Hadoop workload where spark sends a garbage
// request of this kind
//
// GET /testbucket/?list-type=2&delimiter=%2F&max-keys=2&prefix=parquet%2F_SUCCESS%2F&fetch-owner=false
//
// Here spark is expecting that the List() return empty instead, so from MinIO's point
// of view if we simply do a GetObjectInfo() on this prefix by treating it as an object
// We save a lot of calls over the network.
//
// This happens repeatedly for all objects that are created concurrently() avoiding this
// as a List() call is an important performance improvement.
//
// Spark based s3a committers are a big enough use-case to have this optimization.
//
// A sample code to see the improvements is as follows, this sample code is
// simply a read on JSON from MinIO and write it back as "parquet".
//
// import org.apache.spark.sql.SparkSession
// import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
// object SparkJSONRead {
// def main(args: Array[String]): Unit = {
// val spark:SparkSession = SparkSession.builder()
// .appName("SparkByExample")
// .master("local[1]").getOrCreate()
//
// spark.sparkContext.setLogLevel("ERROR")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.endpoint", "http://minio-lb:9000")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.path.style.access", "true")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", "minioadmin")
// spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", "minioadmin")
//
// val df = spark.read.json("s3a://testbucket/s3.json")
//
// df.write.parquet("s3a://testbucket/parquet/")
// }
// }
if matches() {
objInfo, err := z.GetObjectInfo(ctx, bucket, path.Dir(prefix), ObjectOptions{NoLock: true})
if err == nil {
if opts.Lifecycle != nil {
evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, opts.Replication.Config, objInfo)
if evt.Action.Delete() {
globalExpiryState.enqueueByDays(objInfo, evt, lcEventSrc_s3ListObjects)
if !evt.Action.DeleteRestored() {
// Skip entry if ILM action was DeleteVersionAction or DeleteAction
return loi, nil
}
}
}
return loi, nil
}
if isErrBucketNotFound(err) {
return loi, err
}
if contextCanceled(ctx) {
return ListObjectsInfo{}, ctx.Err()
}
}
// Hadoop makes the max-keys=2 listing call just to find if the directory is empty or not, or in the case
// of an object to check for object existence. For versioned buckets, MinIO's non-recursive
// call will report top level prefixes in deleted state, whereas spark/hadoop interpret this as non-empty
// and throw a 404 exception. This is especially a problem for spark jobs overwriting the same partition
// repeatedly. This workaround recursively lists the top 3 entries including delete markers to reflect the
// correct state of the directory in the list results.
opts.Recursive = true
opts.InclDeleted = true
opts.Limit = maxKeys + 1
li, err := listFn(ctx, opts, opts.Limit)
if err == nil {
switch {
case len(li.Objects) == 0 && len(li.Prefixes) == 0:
return loi, nil
case len(li.Objects) > 0 || len(li.Prefixes) > 0:
var o ObjectInfo
var pfx string
if len(li.Objects) > 0 {
o = li.Objects[0]
p := strings.TrimPrefix(o.Name, opts.Prefix)
if p != "" {
sidx := strings.Index(p, "/")
if sidx > 0 {
pfx = p[:sidx]
}
}
}
if o.DeleteMarker {
loi.Objects = append(loi.Objects, ObjectInfo{Bucket: bucket, IsDir: true, Name: prefix})
return loi, nil
} else if len(li.Objects) == 1 {
loi.Objects = append(loi.Objects, o)
loi.Prefixes = append(loi.Prefixes, path.Join(opts.Prefix, pfx))
}
}
return loi, nil
}
}
if len(prefix) > 0 && maxKeys == 1 && marker == "" {
// Optimization for certain applications like
// - Cohesity
// - Actifio, Splunk etc.
// which send ListObjects requests where the actual object
// itself is the prefix and max-keys=1 in such scenarios
// we can simply verify locally if such an object exists
// to avoid the need for ListObjects().
objInfo, err := z.GetObjectInfo(ctx, bucket, prefix, ObjectOptions{NoLock: true})
if err == nil {
if opts.Lifecycle != nil {
evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, opts.Replication.Config, objInfo)
if evt.Action.Delete() {
globalExpiryState.enqueueByDays(objInfo, evt, lcEventSrc_s3ListObjects)
if !evt.Action.DeleteRestored() {
// Skip entry if ILM action was DeleteVersionAction or DeleteAction
return loi, nil
}
}
}
loi.Objects = append(loi.Objects, objInfo)
return loi, nil
}
if isErrBucketNotFound(err) {
return ListObjectsInfo{}, err
}
if contextCanceled(ctx) {
return ListObjectsInfo{}, ctx.Err()
}
}
return listFn(ctx, opts, maxKeys)
}
func (z *erasureServerPools) ListMultipartUploads(ctx context.Context, bucket, prefix, keyMarker, uploadIDMarker, delimiter string, maxUploads int) (ListMultipartsInfo, error) {