diff --git a/app/vmselect/graphite/tags_api.go b/app/vmselect/graphite/tags_api.go
index e1e43cf797..6236aff86d 100644
--- a/app/vmselect/graphite/tags_api.go
+++ b/app/vmselect/graphite/tags_api.go
@@ -192,7 +192,7 @@ func TagsAutoCompleteValuesHandler(startTime time.Time, w http.ResponseWriter, r
 		if err != nil {
 			return err
 		}
-		mns, err := netstorage.SearchMetricNames(nil, sq, deadline)
+		metricNames, err := netstorage.SearchMetricNames(nil, sq, deadline)
 		if err != nil {
 			return fmt.Errorf("cannot fetch metric names for %q: %w", sq, err)
 		}
@@ -200,7 +200,11 @@ func TagsAutoCompleteValuesHandler(startTime time.Time, w http.ResponseWriter, r
 		if tag == "name" {
 			tag = "__name__"
 		}
-		for _, mn := range mns {
+		var mn storage.MetricName
+		for _, metricName := range metricNames {
+			if err := mn.UnmarshalString(metricName); err != nil {
+				return fmt.Errorf("cannot unmarshal metricName=%q: %w", metricName, err)
+			}
 			tagValue := mn.GetTagValue(tag)
 			if len(tagValue) == 0 {
 				continue
@@ -275,12 +279,16 @@ func TagsAutoCompleteTagsHandler(startTime time.Time, w http.ResponseWriter, r *
 		if err != nil {
 			return err
 		}
-		mns, err := netstorage.SearchMetricNames(nil, sq, deadline)
+		metricNames, err := netstorage.SearchMetricNames(nil, sq, deadline)
 		if err != nil {
 			return fmt.Errorf("cannot fetch metric names for %q: %w", sq, err)
 		}
 		m := make(map[string]struct{})
-		for _, mn := range mns {
+		var mn storage.MetricName
+		for _, metricName := range metricNames {
+			if err := mn.UnmarshalString(metricName); err != nil {
+				return fmt.Errorf("cannot unmarshal metricName=%q: %w", metricName, err)
+			}
 			m["name"] = struct{}{}
 			for _, tag := range mn.Tags {
 				m[string(tag.Key)] = struct{}{}
@@ -339,11 +347,14 @@ func TagsFindSeriesHandler(startTime time.Time, w http.ResponseWriter, r *http.R
 	if err != nil {
 		return err
 	}
-	mns, err := netstorage.SearchMetricNames(nil, sq, deadline)
+	metricNames, err := netstorage.SearchMetricNames(nil, sq, deadline)
 	if err != nil {
 		return fmt.Errorf("cannot fetch metric names for %q: %w", sq, err)
 	}
-	paths := getCanonicalPaths(mns)
+	paths, err := getCanonicalPaths(metricNames)
+	if err != nil {
+		return fmt.Errorf("cannot obtain canonical paths: %w", err)
+	}
 	if limit > 0 && limit < len(paths) {
 		paths = paths[:limit]
 	}
@@ -359,14 +370,18 @@ func TagsFindSeriesHandler(startTime time.Time, w http.ResponseWriter, r *http.R
 	return nil
 }
 
-func getCanonicalPaths(mns []storage.MetricName) []string {
-	paths := make([]string, 0, len(mns))
-	for _, mn := range mns {
+func getCanonicalPaths(metricNames []string) ([]string, error) {
+	paths := make([]string, 0, len(metricNames))
+	var mn storage.MetricName
+	for _, metricName := range metricNames {
+		if err := mn.UnmarshalString(metricName); err != nil {
+			return nil, fmt.Errorf("cannot unmarshal metricName=%q: %w", metricName, err)
+		}
 		path := getCanonicalPath(&mn)
 		paths = append(paths, path)
 	}
 	sort.Strings(paths)
-	return paths
+	return paths, nil
 }
 
 func getCanonicalPath(mn *storage.MetricName) string {
diff --git a/app/vmselect/netstorage/netstorage.go b/app/vmselect/netstorage/netstorage.go
index d92d1dee1e..49ca3e1fab 100644
--- a/app/vmselect/netstorage/netstorage.go
+++ b/app/vmselect/netstorage/netstorage.go
@@ -244,12 +244,13 @@ func (rss *Results) RunParallel(qt *querytracer.Tracer, f func(rs *Result, worke
 			// Return just the first error, since other errors are likely duplicate the first error.
 			firstErr = err
 		}
+		rowsReadPerSeries.Update(float64(tsw.rowsProcessed))
 		rowsProcessedTotal += tsw.rowsProcessed
 		putTimeseriesWork(tsw)
 	}
 
-	perQueryRowsProcessed.Update(float64(rowsProcessedTotal))
-	perQuerySeriesProcessed.Update(float64(seriesProcessedTotal))
+	rowsReadPerQuery.Update(float64(rowsProcessedTotal))
+	seriesReadPerQuery.Update(float64(seriesProcessedTotal))
 
 	// Shut down local workers
 	for _, workCh := range workChs {
@@ -261,8 +262,11 @@ func (rss *Results) RunParallel(qt *querytracer.Tracer, f func(rs *Result, worke
 	return firstErr
 }
 
-var perQueryRowsProcessed = metrics.NewHistogram(`vm_per_query_rows_processed_count`)
-var perQuerySeriesProcessed = metrics.NewHistogram(`vm_per_query_series_processed_count`)
+var (
+	rowsReadPerSeries  = metrics.NewHistogram(`vm_rows_read_per_series`)
+	rowsReadPerQuery   = metrics.NewHistogram(`vm_rows_read_per_query`)
+	seriesReadPerQuery = metrics.NewHistogram(`vm_series_read_per_query`)
+)
 
 var gomaxprocs = cgroup.AvailableCPUs()
 
@@ -1010,7 +1014,9 @@ var exportWorkPool = &sync.Pool{
 }
 
 // SearchMetricNames returns all the metric names matching sq until the given deadline.
-func SearchMetricNames(qt *querytracer.Tracer, sq *storage.SearchQuery, deadline searchutils.Deadline) ([]storage.MetricName, error) {
+//
+// The returned metric names must be unmarshaled via storage.MetricName.UnmarshalString().
+func SearchMetricNames(qt *querytracer.Tracer, sq *storage.SearchQuery, deadline searchutils.Deadline) ([]string, error) {
 	qt = qt.NewChild("fetch metric names: %s", sq)
 	defer qt.Done()
 	if deadline.Exceeded() {
@@ -1030,11 +1036,13 @@ func SearchMetricNames(qt *querytracer.Tracer, sq *storage.SearchQuery, deadline
 		return nil, err
 	}
 
-	mns, err := vmstorage.SearchMetricNames(qt, tfss, tr, sq.MaxMetrics, deadline.Deadline())
+	metricNames, err := vmstorage.SearchMetricNames(qt, tfss, tr, sq.MaxMetrics, deadline.Deadline())
 	if err != nil {
 		return nil, fmt.Errorf("cannot find metric names: %w", err)
 	}
-	return mns, nil
+	sort.Strings(metricNames)
+	qt.Printf("sort %d metric names", len(metricNames))
+	return metricNames, nil
 }
 
 // ProcessSearchQuery performs sq until the given deadline.
diff --git a/app/vmselect/prometheus/prometheus.go b/app/vmselect/prometheus/prometheus.go
index 3aceb4c35d..26b59b9ac4 100644
--- a/app/vmselect/prometheus/prometheus.go
+++ b/app/vmselect/prometheus/prometheus.go
@@ -48,7 +48,7 @@ var (
 	maxFederateSeries   = flag.Int("search.maxFederateSeries", 1e6, "The maximum number of time series, which can be returned from /federate. This option allows limiting memory usage")
 	maxExportSeries     = flag.Int("search.maxExportSeries", 10e6, "The maximum number of time series, which can be returned from /api/v1/export* APIs. This option allows limiting memory usage")
 	maxTSDBStatusSeries = flag.Int("search.maxTSDBStatusSeries", 10e6, "The maximum number of time series, which can be processed during the call to /api/v1/status/tsdb. This option allows limiting memory usage")
-	maxSeriesLimit      = flag.Int("search.maxSeries", 100e3, "The maximum number of time series, which can be returned from /api/v1/series. This option allows limiting memory usage")
+	maxSeriesLimit      = flag.Int("search.maxSeries", 30e3, "The maximum number of time series, which can be returned from /api/v1/series. This option allows limiting memory usage")
 )
 
 // Default step used if not set.
@@ -610,7 +610,7 @@ func SeriesHandler(qt *querytracer.Tracer, startTime time.Time, w http.ResponseW
 		cp.start = cp.end - defaultStep
 	}
 	sq := storage.NewSearchQuery(cp.start, cp.end, cp.filterss, *maxSeriesLimit)
-	mns, err := netstorage.SearchMetricNames(qt, sq, cp.deadline)
+	metricNames, err := netstorage.SearchMetricNames(qt, sq, cp.deadline)
 	if err != nil {
 		return fmt.Errorf("cannot fetch time series for %q: %w", sq, err)
 	}
@@ -620,7 +620,7 @@ func SeriesHandler(qt *querytracer.Tracer, startTime time.Time, w http.ResponseW
 	qtDone := func() {
 		qt.Donef("start=%d, end=%d", cp.start, cp.end)
 	}
-	WriteSeriesResponse(bw, mns, qt, qtDone)
+	WriteSeriesResponse(bw, metricNames, qt, qtDone)
 	if err := bw.Flush(); err != nil {
 		return err
 	}
diff --git a/app/vmselect/prometheus/series_response.qtpl b/app/vmselect/prometheus/series_response.qtpl
index 9ca28e5481..2584859986 100644
--- a/app/vmselect/prometheus/series_response.qtpl
+++ b/app/vmselect/prometheus/series_response.qtpl
@@ -6,17 +6,23 @@
 {% stripspace %}
 SeriesResponse generates response for /api/v1/series.
 See https://prometheus.io/docs/prometheus/latest/querying/api/#finding-series-by-label-matchers
-{% func SeriesResponse(mns []storage.MetricName, qt *querytracer.Tracer, qtDone func()) %}
+{% func SeriesResponse(metricNames []string, qt *querytracer.Tracer, qtDone func()) %}
 {
 	"status":"success",
 	"data":[
-		{% for i := range mns %}
-			{%= metricNameObject(&mns[i]) %}
-			{% if i+1 < len(mns) %},{% endif %}
+		{% code var mn storage.MetricName %}
+		{% for i, metricName := range metricNames %}
+			{% code err := mn.UnmarshalString(metricName) %}
+			{% if err != nil %}
+				{%q= err.Error() %}
+			{% else %}
+				{%= metricNameObject(&mn) %}
+			{% endif %}
+			{% if i+1 < len(metricNames) %},{% endif %}
 		{% endfor %}
 	]
 	{% code
-		qt.Printf("generate response: series=%d", len(mns))
+		qt.Printf("generate response: series=%d", len(metricNames))
 		qtDone()
 	%}
 	{%= dumpQueryTrace(qt) %}
diff --git a/app/vmselect/prometheus/series_response.qtpl.go b/app/vmselect/prometheus/series_response.qtpl.go
index e2b1eeb62a..1ea12ff1d0 100644
--- a/app/vmselect/prometheus/series_response.qtpl.go
+++ b/app/vmselect/prometheus/series_response.qtpl.go
@@ -26,56 +26,70 @@ var (
 )
 
 //line app/vmselect/prometheus/series_response.qtpl:9
-func StreamSeriesResponse(qw422016 *qt422016.Writer, mns []storage.MetricName, qt *querytracer.Tracer, qtDone func()) {
+func StreamSeriesResponse(qw422016 *qt422016.Writer, metricNames []string, qt *querytracer.Tracer, qtDone func()) {
 //line app/vmselect/prometheus/series_response.qtpl:9
 	qw422016.N().S(`{"status":"success","data":[`)
 //line app/vmselect/prometheus/series_response.qtpl:13
-	for i := range mns {
+	var mn storage.MetricName
+
 //line app/vmselect/prometheus/series_response.qtpl:14
-		streammetricNameObject(qw422016, &mns[i])
+	for i, metricName := range metricNames {
 //line app/vmselect/prometheus/series_response.qtpl:15
-		if i+1 < len(mns) {
-//line app/vmselect/prometheus/series_response.qtpl:15
-			qw422016.N().S(`,`)
-//line app/vmselect/prometheus/series_response.qtpl:15
-		}
+		err := mn.UnmarshalString(metricName)
+
 //line app/vmselect/prometheus/series_response.qtpl:16
-	}
-//line app/vmselect/prometheus/series_response.qtpl:16
-	qw422016.N().S(`]`)
+		if err != nil {
+//line app/vmselect/prometheus/series_response.qtpl:17
+			qw422016.N().Q(err.Error())
+//line app/vmselect/prometheus/series_response.qtpl:18
+		} else {
 //line app/vmselect/prometheus/series_response.qtpl:19
-	qt.Printf("generate response: series=%d", len(mns))
+			streammetricNameObject(qw422016, &mn)
+//line app/vmselect/prometheus/series_response.qtpl:20
+		}
+//line app/vmselect/prometheus/series_response.qtpl:21
+		if i+1 < len(metricNames) {
+//line app/vmselect/prometheus/series_response.qtpl:21
+			qw422016.N().S(`,`)
+//line app/vmselect/prometheus/series_response.qtpl:21
+		}
+//line app/vmselect/prometheus/series_response.qtpl:22
+	}
+//line app/vmselect/prometheus/series_response.qtpl:22
+	qw422016.N().S(`]`)
+//line app/vmselect/prometheus/series_response.qtpl:25
+	qt.Printf("generate response: series=%d", len(metricNames))
 	qtDone()
 
-//line app/vmselect/prometheus/series_response.qtpl:22
+//line app/vmselect/prometheus/series_response.qtpl:28
 	streamdumpQueryTrace(qw422016, qt)
-//line app/vmselect/prometheus/series_response.qtpl:22
+//line app/vmselect/prometheus/series_response.qtpl:28
 	qw422016.N().S(`}`)
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
 }
 
-//line app/vmselect/prometheus/series_response.qtpl:24
-func WriteSeriesResponse(qq422016 qtio422016.Writer, mns []storage.MetricName, qt *querytracer.Tracer, qtDone func()) {
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
+func WriteSeriesResponse(qq422016 qtio422016.Writer, metricNames []string, qt *querytracer.Tracer, qtDone func()) {
+//line app/vmselect/prometheus/series_response.qtpl:30
 	qw422016 := qt422016.AcquireWriter(qq422016)
-//line app/vmselect/prometheus/series_response.qtpl:24
-	StreamSeriesResponse(qw422016, mns, qt, qtDone)
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
+	StreamSeriesResponse(qw422016, metricNames, qt, qtDone)
+//line app/vmselect/prometheus/series_response.qtpl:30
 	qt422016.ReleaseWriter(qw422016)
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
 }
 
-//line app/vmselect/prometheus/series_response.qtpl:24
-func SeriesResponse(mns []storage.MetricName, qt *querytracer.Tracer, qtDone func()) string {
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
+func SeriesResponse(metricNames []string, qt *querytracer.Tracer, qtDone func()) string {
+//line app/vmselect/prometheus/series_response.qtpl:30
 	qb422016 := qt422016.AcquireByteBuffer()
-//line app/vmselect/prometheus/series_response.qtpl:24
-	WriteSeriesResponse(qb422016, mns, qt, qtDone)
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
+	WriteSeriesResponse(qb422016, metricNames, qt, qtDone)
+//line app/vmselect/prometheus/series_response.qtpl:30
 	qs422016 := string(qb422016.B)
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
 	qt422016.ReleaseByteBuffer(qb422016)
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
 	return qs422016
-//line app/vmselect/prometheus/series_response.qtpl:24
+//line app/vmselect/prometheus/series_response.qtpl:30
 }
diff --git a/app/vmselect/promql/eval.go b/app/vmselect/promql/eval.go
index 92836df950..23b0036135 100644
--- a/app/vmselect/promql/eval.go
+++ b/app/vmselect/promql/eval.go
@@ -8,6 +8,7 @@ import (
 	"sort"
 	"strings"
 	"sync"
+	"sync/atomic"
 
 	"github.com/VictoriaMetrics/VictoriaMetrics/app/vmselect/netstorage"
 	"github.com/VictoriaMetrics/VictoriaMetrics/app/vmselect/searchutils"
@@ -762,30 +763,36 @@ func evalRollupFuncWithSubquery(qt *querytracer.Tracer, ec *EvalConfig, funcName
 	}
 	tss := make([]*timeseries, 0, len(tssSQ)*len(rcs))
 	var tssLock sync.Mutex
+	var samplesScannedTotal uint64
 	keepMetricNames := getKeepMetricNames(expr)
 	doParallel(tssSQ, func(tsSQ *timeseries, values []float64, timestamps []int64) ([]float64, []int64) {
 		values, timestamps = removeNanValues(values[:0], timestamps[:0], tsSQ.Values, tsSQ.Timestamps)
 		preFunc(values, timestamps)
 		for _, rc := range rcs {
 			if tsm := newTimeseriesMap(funcName, keepMetricNames, sharedTimestamps, &tsSQ.MetricName); tsm != nil {
-				rc.DoTimeseriesMap(tsm, values, timestamps)
+				samplesScanned := rc.DoTimeseriesMap(tsm, values, timestamps)
+				atomic.AddUint64(&samplesScannedTotal, samplesScanned)
 				tssLock.Lock()
 				tss = tsm.AppendTimeseriesTo(tss)
 				tssLock.Unlock()
 				continue
 			}
 			var ts timeseries
-			doRollupForTimeseries(funcName, keepMetricNames, rc, &ts, &tsSQ.MetricName, values, timestamps, sharedTimestamps)
+			samplesScanned := doRollupForTimeseries(funcName, keepMetricNames, rc, &ts, &tsSQ.MetricName, values, timestamps, sharedTimestamps)
+			atomic.AddUint64(&samplesScannedTotal, samplesScanned)
 			tssLock.Lock()
 			tss = append(tss, &ts)
 			tssLock.Unlock()
 		}
 		return values, timestamps
 	})
-	qt.Printf("rollup %s() over %d series returned by subquery: series=%d", funcName, len(tssSQ), len(tss))
+	rowsScannedPerQuery.Update(float64(samplesScannedTotal))
+	qt.Printf("rollup %s() over %d series returned by subquery: series=%d, samplesScanned=%d", funcName, len(tssSQ), len(tss), samplesScannedTotal)
 	return tss, nil
 }
 
+var rowsScannedPerQuery = metrics.NewHistogram(`vm_rows_scanned_per_query`)
+
 func getKeepMetricNames(expr metricsql.Expr) bool {
 	if ae, ok := expr.(*metricsql.AggrFuncExpr); ok {
 		// Extract rollupFunc(...) from aggrFunc(rollupFunc(...)).
@@ -981,8 +988,9 @@ func getRollupMemoryLimiter() *memoryLimiter {
 func evalRollupWithIncrementalAggregate(qt *querytracer.Tracer, funcName string, keepMetricNames bool,
 	iafc *incrementalAggrFuncContext, rss *netstorage.Results, rcs []*rollupConfig,
 	preFunc func(values []float64, timestamps []int64), sharedTimestamps []int64) ([]*timeseries, error) {
-	qt = qt.NewChild("rollup %s() with incremental aggregation %s() over %d series", funcName, iafc.ae.Name, rss.Len())
+	qt = qt.NewChild("rollup %s() with incremental aggregation %s() over %d series; rollupConfigs=%s", funcName, iafc.ae.Name, rss.Len(), rcs)
 	defer qt.Done()
+	var samplesScannedTotal uint64
 	err := rss.RunParallel(qt, func(rs *netstorage.Result, workerID uint) error {
 		rs.Values, rs.Timestamps = dropStaleNaNs(funcName, rs.Values, rs.Timestamps)
 		preFunc(rs.Values, rs.Timestamps)
@@ -990,14 +998,16 @@ func evalRollupWithIncrementalAggregate(qt *querytracer.Tracer, funcName string,
 		defer putTimeseries(ts)
 		for _, rc := range rcs {
 			if tsm := newTimeseriesMap(funcName, keepMetricNames, sharedTimestamps, &rs.MetricName); tsm != nil {
-				rc.DoTimeseriesMap(tsm, rs.Values, rs.Timestamps)
+				samplesScanned := rc.DoTimeseriesMap(tsm, rs.Values, rs.Timestamps)
 				for _, ts := range tsm.m {
 					iafc.updateTimeseries(ts, workerID)
 				}
+				atomic.AddUint64(&samplesScannedTotal, samplesScanned)
 				continue
 			}
 			ts.Reset()
-			doRollupForTimeseries(funcName, keepMetricNames, rc, ts, &rs.MetricName, rs.Values, rs.Timestamps, sharedTimestamps)
+			samplesScanned := doRollupForTimeseries(funcName, keepMetricNames, rc, ts, &rs.MetricName, rs.Values, rs.Timestamps, sharedTimestamps)
+			atomic.AddUint64(&samplesScannedTotal, samplesScanned)
 			iafc.updateTimeseries(ts, workerID)
 
 			// ts.Timestamps points to sharedTimestamps. Zero it, so it can be re-used.
@@ -1010,29 +1020,33 @@ func evalRollupWithIncrementalAggregate(qt *querytracer.Tracer, funcName string,
 		return nil, err
 	}
 	tss := iafc.finalizeTimeseries()
-	qt.Printf("series after aggregation with %s(): %d", iafc.ae.Name, len(tss))
+	rowsScannedPerQuery.Update(float64(samplesScannedTotal))
+	qt.Printf("series after aggregation with %s(): %d; samplesScanned=%d", iafc.ae.Name, len(tss), samplesScannedTotal)
 	return tss, nil
 }
 
 func evalRollupNoIncrementalAggregate(qt *querytracer.Tracer, funcName string, keepMetricNames bool, rss *netstorage.Results, rcs []*rollupConfig,
 	preFunc func(values []float64, timestamps []int64), sharedTimestamps []int64) ([]*timeseries, error) {
-	qt = qt.NewChild("rollup %s() over %d series", funcName, rss.Len())
+	qt = qt.NewChild("rollup %s() over %d series; rollupConfigs=%s", funcName, rss.Len(), rcs)
 	defer qt.Done()
 	tss := make([]*timeseries, 0, rss.Len()*len(rcs))
 	var tssLock sync.Mutex
+	var samplesScannedTotal uint64
 	err := rss.RunParallel(qt, func(rs *netstorage.Result, workerID uint) error {
 		rs.Values, rs.Timestamps = dropStaleNaNs(funcName, rs.Values, rs.Timestamps)
 		preFunc(rs.Values, rs.Timestamps)
 		for _, rc := range rcs {
 			if tsm := newTimeseriesMap(funcName, keepMetricNames, sharedTimestamps, &rs.MetricName); tsm != nil {
-				rc.DoTimeseriesMap(tsm, rs.Values, rs.Timestamps)
+				samplesScanned := rc.DoTimeseriesMap(tsm, rs.Values, rs.Timestamps)
+				atomic.AddUint64(&samplesScannedTotal, samplesScanned)
 				tssLock.Lock()
 				tss = tsm.AppendTimeseriesTo(tss)
 				tssLock.Unlock()
 				continue
 			}
 			var ts timeseries
-			doRollupForTimeseries(funcName, keepMetricNames, rc, &ts, &rs.MetricName, rs.Values, rs.Timestamps, sharedTimestamps)
+			samplesScanned := doRollupForTimeseries(funcName, keepMetricNames, rc, &ts, &rs.MetricName, rs.Values, rs.Timestamps, sharedTimestamps)
+			atomic.AddUint64(&samplesScannedTotal, samplesScanned)
 			tssLock.Lock()
 			tss = append(tss, &ts)
 			tssLock.Unlock()
@@ -1042,11 +1056,13 @@ func evalRollupNoIncrementalAggregate(qt *querytracer.Tracer, funcName string, k
 	if err != nil {
 		return nil, err
 	}
+	rowsScannedPerQuery.Update(float64(samplesScannedTotal))
+	qt.Printf("samplesScanned=%d", samplesScannedTotal)
 	return tss, nil
 }
 
 func doRollupForTimeseries(funcName string, keepMetricNames bool, rc *rollupConfig, tsDst *timeseries, mnSrc *storage.MetricName,
-	valuesSrc []float64, timestampsSrc []int64, sharedTimestamps []int64) {
+	valuesSrc []float64, timestampsSrc []int64, sharedTimestamps []int64) uint64 {
 	tsDst.MetricName.CopyFrom(mnSrc)
 	if len(rc.TagValue) > 0 {
 		tsDst.MetricName.AddTag("rollup", rc.TagValue)
@@ -1054,9 +1070,11 @@ func doRollupForTimeseries(funcName string, keepMetricNames bool, rc *rollupConf
 	if !keepMetricNames && !rollupFuncsKeepMetricName[funcName] {
 		tsDst.MetricName.ResetMetricGroup()
 	}
-	tsDst.Values = rc.Do(tsDst.Values[:0], valuesSrc, timestampsSrc)
+	var samplesScanned uint64
+	tsDst.Values, samplesScanned = rc.Do(tsDst.Values[:0], valuesSrc, timestampsSrc)
 	tsDst.Timestamps = sharedTimestamps
 	tsDst.denyReuse = true
+	return samplesScanned
 }
 
 var bbPool bytesutil.ByteBufferPool
diff --git a/app/vmselect/promql/rollup.go b/app/vmselect/promql/rollup.go
index 640a35828e..b19ac82735 100644
--- a/app/vmselect/promql/rollup.go
+++ b/app/vmselect/promql/rollup.go
@@ -416,6 +416,12 @@ type rollupConfig struct {
 	isDefaultRollup bool
 }
 
+func (rc *rollupConfig) String() string {
+	start := storage.TimestampToHumanReadableFormat(rc.Start)
+	end := storage.TimestampToHumanReadableFormat(rc.End)
+	return fmt.Sprintf("timeRange=[%s..%s], step=%d, window=%d, points=%d", start, end, rc.Step, rc.Window, len(rc.Timestamps))
+}
+
 var (
 	nan = math.NaN()
 	inf = math.Inf(1)
@@ -483,18 +489,20 @@ func (tsm *timeseriesMap) GetOrCreateTimeseries(labelName, labelValue string) *t
 // timestamps must cover time range [rc.Start - rc.Window - maxSilenceInterval ... rc.End].
 //
 // Do cannot be called from concurrent goroutines.
-func (rc *rollupConfig) Do(dstValues []float64, values []float64, timestamps []int64) []float64 {
+func (rc *rollupConfig) Do(dstValues []float64, values []float64, timestamps []int64) ([]float64, uint64) {
 	return rc.doInternal(dstValues, nil, values, timestamps)
 }
 
 // DoTimeseriesMap calculates rollups for the given timestamps and values and puts them to tsm.
-func (rc *rollupConfig) DoTimeseriesMap(tsm *timeseriesMap, values []float64, timestamps []int64) {
+func (rc *rollupConfig) DoTimeseriesMap(tsm *timeseriesMap, values []float64, timestamps []int64) uint64 {
 	ts := getTimeseries()
-	ts.Values = rc.doInternal(ts.Values[:0], tsm, values, timestamps)
+	var samplesScanned uint64
+	ts.Values, samplesScanned = rc.doInternal(ts.Values[:0], tsm, values, timestamps)
 	putTimeseries(ts)
+	return samplesScanned
 }
 
-func (rc *rollupConfig) doInternal(dstValues []float64, tsm *timeseriesMap, values []float64, timestamps []int64) []float64 {
+func (rc *rollupConfig) doInternal(dstValues []float64, tsm *timeseriesMap, values []float64, timestamps []int64) ([]float64, uint64) {
 	// Sanity checks.
 	if rc.Step <= 0 {
 		logger.Panicf("BUG: Step must be bigger than 0; got %d", rc.Step)
@@ -544,6 +552,7 @@ func (rc *rollupConfig) doInternal(dstValues []float64, tsm *timeseriesMap, valu
 	ni := 0
 	nj := 0
 	f := rc.Func
+	var samplesScanned uint64
 	for _, tEnd := range rc.Timestamps {
 		tStart := tEnd - window
 		ni = seekFirstTimestampIdxAfter(timestamps[i:], tStart, ni)
@@ -575,11 +584,12 @@ func (rc *rollupConfig) doInternal(dstValues []float64, tsm *timeseriesMap, valu
 		rfa.currTimestamp = tEnd
 		value := f(rfa)
 		rfa.idx++
+		samplesScanned += uint64(len(rfa.values))
 		dstValues = append(dstValues, value)
 	}
 	putRollupFuncArg(rfa)
 
-	return dstValues
+	return dstValues, samplesScanned
 }
 
 func seekFirstTimestampIdxAfter(timestamps []int64, seekTimestamp int64, nHint int) int {
diff --git a/app/vmselect/promql/rollup_test.go b/app/vmselect/promql/rollup_test.go
index 67e0934cfa..bfc6d670ef 100644
--- a/app/vmselect/promql/rollup_test.go
+++ b/app/vmselect/promql/rollup_test.go
@@ -584,7 +584,10 @@ func TestRollupNoWindowNoPoints(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned != 0 {
+			t.Fatalf("expecting zero samplesScanned from rollupConfig.Do; got %d", samplesScanned)
+		}
 		valuesExpected := []float64{nan, nan, nan, nan, nan}
 		timestampsExpected := []int64{0, 1, 2, 3, 4}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -598,7 +601,10 @@ func TestRollupNoWindowNoPoints(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{2, 0, 0, 0, nan, nan, nan, nan}
 		timestampsExpected := []int64{120, 124, 128, 132, 136, 140, 144, 148}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -615,7 +621,10 @@ func TestRollupWindowNoPoints(t *testing.T) {
 			Window: 3,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned != 0 {
+			t.Fatalf("expecting zero samplesScanned from rollupConfig.Do; got %d", samplesScanned)
+		}
 		valuesExpected := []float64{nan, nan, nan, nan, nan}
 		timestampsExpected := []int64{0, 1, 2, 3, 4}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -629,7 +638,10 @@ func TestRollupWindowNoPoints(t *testing.T) {
 			Window: 3,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned != 0 {
+			t.Fatalf("expecting zero samplesScanned from rollupConfig.Do; got %d", samplesScanned)
+		}
 		valuesExpected := []float64{nan, nan, nan, nan}
 		timestampsExpected := []int64{161, 171, 181, 191}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -646,7 +658,10 @@ func TestRollupNoWindowPartialPoints(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 123, nan, 34, nan, 44}
 		timestampsExpected := []int64{0, 5, 10, 15, 20, 25}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -660,7 +675,10 @@ func TestRollupNoWindowPartialPoints(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{44, 32, 34, nan}
 		timestampsExpected := []int64{100, 120, 140, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -674,7 +692,10 @@ func TestRollupNoWindowPartialPoints(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, nan, 123, 34, 32}
 		timestampsExpected := []int64{-50, 0, 50, 100, 150}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -691,7 +712,10 @@ func TestRollupWindowPartialPoints(t *testing.T) {
 			Window: 8,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 123, 123, 34, 34}
 		timestampsExpected := []int64{0, 5, 10, 15, 20}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -705,7 +729,10 @@ func TestRollupWindowPartialPoints(t *testing.T) {
 			Window: 18,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{44, 34, 34, nan}
 		timestampsExpected := []int64{100, 120, 140, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -719,7 +746,10 @@ func TestRollupWindowPartialPoints(t *testing.T) {
 			Window: 19,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 54, 44, nan}
 		timestampsExpected := []int64{0, 50, 100, 150}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -736,7 +766,10 @@ func TestRollupFuncsLookbackDelta(t *testing.T) {
 			LookbackDelta: 1,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{99, nan, 44, nan, 32, 34, nan}
 		timestampsExpected := []int64{80, 90, 100, 110, 120, 130, 140}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -750,7 +783,10 @@ func TestRollupFuncsLookbackDelta(t *testing.T) {
 			LookbackDelta: 7,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{99, nan, 44, nan, 32, 34, nan}
 		timestampsExpected := []int64{80, 90, 100, 110, 120, 130, 140}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -764,7 +800,10 @@ func TestRollupFuncsLookbackDelta(t *testing.T) {
 			LookbackDelta: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{99, nan, 44, nan, 32, 34, nan}
 		timestampsExpected := []int64{80, 90, 100, 110, 120, 130, 140}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -781,7 +820,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 123, 54, 44, 34}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -795,7 +837,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 4, 4, 3, 1}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -809,7 +854,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 21, 12, 32, 34}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -823,7 +871,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 123, 99, 44, 34}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -837,7 +888,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 222, 199, 110, 34}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -851,7 +905,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 21, -9, 22, 0}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -865,7 +922,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, -102, -42, -10, nan}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -879,7 +939,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{123, 33, -87, 0}
 		timestampsExpected := []int64{10, 50, 90, 130}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -893,7 +956,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 0.004, 0, 0, 0.03}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -907,7 +973,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 0.031, 0.044, 0.04, 0.01}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -921,7 +990,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 200,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 0.031, 0.075, 0.115, 0.125}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -935,7 +1007,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 0.010333333333333333, 0.011, 0.013333333333333334, 0.01}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -949,7 +1024,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 80,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 0.010333333333333333, 0.010714285714285714, 0.012, 0.0125}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -963,7 +1041,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 4, 4, 3, 0}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -977,7 +1058,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 3, 3, 2, 0}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -991,7 +1075,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 9,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 1, 1, 1, 1, 0}
 		timestampsExpected := []int64{0, 9, 18, 27, 36, 45}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1005,7 +1092,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 2, 2, 1, 0}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1019,7 +1109,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 55.5, 49.75, 36.666666666666664, 34}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1033,7 +1126,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, -2879.310344827588, 127.87627310448904, -496.5831435079728, 0}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1047,7 +1143,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, nan, nan, 0, -8900, 0}
 		timestampsExpected := []int64{0, 4, 8, 12, 16, 20}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1061,7 +1160,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, -1916.6666666666665, -43500, 400, 0}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1075,7 +1177,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 39.81519810323691, 32.080952292598795, 5.2493385826745405, 0}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1089,7 +1194,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 2.148, 1.593, 1.156, 1.36}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1103,7 +1211,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 0,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 4, 4, 3, 1}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1117,7 +1228,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 80,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 4, 7, 6, 3}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1131,7 +1245,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 80,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 21, 34, 34, 34}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1145,7 +1262,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 80,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, 2775, 5262.5, 3678.5714285714284, 2880}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1159,7 +1279,10 @@ func TestRollupFuncsNoWindow(t *testing.T) {
 			Window: 80,
 		}
 		rc.Timestamps = getTimestamps(rc.Start, rc.End, rc.Step)
-		values := rc.Do(nil, testValues, testTimestamps)
+		values, samplesScanned := rc.Do(nil, testValues, testTimestamps)
+		if samplesScanned == 0 {
+			t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+		}
 		valuesExpected := []float64{nan, -0.86650328627136, -1.1200838283548589, -0.40035755084856683, nan}
 		timestampsExpected := []int64{0, 40, 80, 120, 160}
 		testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
@@ -1181,7 +1304,10 @@ func TestRollupBigNumberOfValues(t *testing.T) {
 		srcValues[i] = float64(i)
 		srcTimestamps[i] = int64(i / 2)
 	}
-	values := rc.Do(nil, srcValues, srcTimestamps)
+	values, samplesScanned := rc.Do(nil, srcValues, srcTimestamps)
+	if samplesScanned == 0 {
+		t.Fatalf("expecting non-zero samplesScanned from rollupConfig.Do")
+	}
 	valuesExpected := []float64{1, 4001, 8001, 9999, nan, nan}
 	timestampsExpected := []int64{0, 2000, 4000, 6000, 8000, 10000}
 	testRowsEqual(t, values, rc.Timestamps, valuesExpected, timestampsExpected)
diff --git a/app/vmstorage/main.go b/app/vmstorage/main.go
index 2bcb6e6b81..d2f1695b68 100644
--- a/app/vmstorage/main.go
+++ b/app/vmstorage/main.go
@@ -176,11 +176,11 @@ func DeleteMetrics(qt *querytracer.Tracer, tfss []*storage.TagFilters) (int, err
 }
 
 // SearchMetricNames returns metric names for the given tfss on the given tr.
-func SearchMetricNames(qt *querytracer.Tracer, tfss []*storage.TagFilters, tr storage.TimeRange, maxMetrics int, deadline uint64) ([]storage.MetricName, error) {
+func SearchMetricNames(qt *querytracer.Tracer, tfss []*storage.TagFilters, tr storage.TimeRange, maxMetrics int, deadline uint64) ([]string, error) {
 	WG.Add(1)
-	mns, err := Storage.SearchMetricNames(qt, tfss, tr, maxMetrics, deadline)
+	metricNames, err := Storage.SearchMetricNames(qt, tfss, tr, maxMetrics, deadline)
 	WG.Done()
-	return mns, err
+	return metricNames, err
 }
 
 // SearchLabelNamesWithFiltersOnTimeRange searches for tag keys matching the given tfss on tr.
diff --git a/docs/CHANGELOG.md b/docs/CHANGELOG.md
index 6b6d5ff743..b42696deb2 100644
--- a/docs/CHANGELOG.md
+++ b/docs/CHANGELOG.md
@@ -33,6 +33,13 @@ scrape_configs:
 ```
 
 * FEATURE: [query tracing](https://docs.victoriametrics.com/Single-server-VictoriaMetrics.html#query-tracing): show timestamps in query traces in human-readable format (aka `RFC3339` in UTC timezone) instead of milliseconds since Unix epoch. For example, `2022-06-27T10:32:54.506Z` instead of `1656325974506`.
+* FEATURE: improve performance of [/api/v1/series](https://prometheus.io/docs/prometheus/latest/querying/api/#finding-series-by-label-matchers) requests, which return big number of time series.
+* FEATURE: expose additional histogram metrics at `http://victoriametrics:8428/metrics`, which may help understanding query workload:
+
+  * `vm_rows_read_per_query` - the number of raw samples read per query.
+  * `vm_rows_scanned_per_query` - the number of raw samples scanned per query. This number can exceed `vm_rows_read_per_query` if `step` query arg passed to [/api/v1/query_range](https://prometheus.io/docs/prometheus/latest/querying/api/#range-queries) is smaller than the lookbehind window set in square brackets of [rollup function](https://docs.victoriametrics.com/MetricsQL.html#rollup-functions). For example, if `increase(some_metric[1h])` is executed with the `step=5m`, then the same raw samples on a hour time range are scanned `1h/5m=12` times. See [this article](https://valyala.medium.com/how-to-optimize-promql-and-metricsql-queries-85a1b75bf986) for details.
+  * `vm_rows_read_per_series` - the number of raw samples read per queried series.
+  * `vm_series_read_per_query` - the number of series read per query.
 
 * BUGFIX: [vmalert](https://docs.victoriametrics.com/vmalert.html): allow using `__name__` label (aka [metric name](https://prometheus.io/docs/prometheus/latest/querying/basics/#time-series-selectors)) in alerting annotations. For example `{{ $labels.__name__ }}: Too high connection number for "{{ $labels.instance }}`.
 * BUGFIX: limit max memory occupied by the cache, which stores parsed regular expressions. Previously too long regular expressions passed in [MetricsQL queries](https://docs.victoriametrics.com/MetricsQL.html) could result in big amounts of used memory (e.g. multiple of gigabytes). Now the max cache size for parsed regexps is limited to a a few megabytes.
diff --git a/docs/keyConcepts.md b/docs/keyConcepts.md
index 6d893228be..858bd012fd 100644
--- a/docs/keyConcepts.md
+++ b/docs/keyConcepts.md
@@ -131,36 +131,36 @@ functions used with gauges are [aggregation and grouping functions](#aggregation
 Histogram is a set of [counter](#counter) metrics with different labels for tracking the dispersion
 and [quantiles](https://prometheus.io/docs/practices/histograms/#quantiles) of the observed value. For example, in
 VictoriaMetrics we track how many rows is processed per query using the histogram with the
-name `vm_per_query_rows_processed_count`. The exposition format for this histogram has the following form:
+name `vm_rows_read_per_query`. The exposition format for this histogram has the following form:
 
 ```
-vm_per_query_rows_processed_count_bucket{vmrange="4.084e+02...4.642e+02"} 2
-vm_per_query_rows_processed_count_bucket{vmrange="5.275e+02...5.995e+02"} 1
-vm_per_query_rows_processed_count_bucket{vmrange="8.799e+02...1.000e+03"} 1
-vm_per_query_rows_processed_count_bucket{vmrange="1.468e+03...1.668e+03"} 3
-vm_per_query_rows_processed_count_bucket{vmrange="1.896e+03...2.154e+03"} 4
-vm_per_query_rows_processed_count_sum 15582
-vm_per_query_rows_processed_count_count 11
+vm_rows_read_per_query_bucket{vmrange="4.084e+02...4.642e+02"} 2
+vm_rows_read_per_query_bucket{vmrange="5.275e+02...5.995e+02"} 1
+vm_rows_read_per_query_bucket{vmrange="8.799e+02...1.000e+03"} 1
+vm_rows_read_per_query_bucket{vmrange="1.468e+03...1.668e+03"} 3
+vm_rows_read_per_query_bucket{vmrange="1.896e+03...2.154e+03"} 4
+vm_rows_read_per_query_sum 15582
+vm_rows_read_per_query_count 11
 ```
 
-In practice, histogram `vm_per_query_rows_processed_count` may be used in the following way:
+In practice, histogram `vm_rows_read_per_query` may be used in the following way:
 
 ```go
 // define the histogram
-perQueryRowsProcessed := metrics.NewHistogram(`vm_per_query_rows_processed_count`)
+rowsReadPerQuery := metrics.NewHistogram(`vm_rows_read_per_query`)
 
 // use the histogram during processing
 for _, query := range queries {
-    perQueryRowsProcessed.Update(len(query.Rows))
+    rowsReadPerQuery.Update(float64(len(query.Rows)))
 }
 ```
 
-Now let's see what happens each time when `perQueryRowsProcessed.Update` is called:
+Now let's see what happens each time when `rowsReadPerQuery.Update` is called:
 
-* counter `vm_per_query_rows_processed_count_sum` increments by value of `len(query.Rows)` expression and accounts for
+* counter `vm_rows_read_per_query_sum` increments by value of `len(query.Rows)` expression and accounts for
   total sum of all observed values;
-* counter `vm_per_query_rows_processed_count_count` increments by 1 and accounts for total number of observations;
-* counter `vm_per_query_rows_processed_count_bucket` gets incremented only if observed value is within the
+* counter `vm_rows_read_per_query_count` increments by 1 and accounts for total number of observations;
+* counter `vm_rows_read_per_query_bucket` gets incremented only if observed value is within the
   range (`bucket`) defined in `vmrange`.
 
 Such a combination of `counter` metrics allows
@@ -823,4 +823,4 @@ details [here](https://docs.victoriametrics.com/Single-server-VictoriaMetrics.ht
 ### Deduplication
 
 VictoriaMetrics supports data points deduplication after data was written to the storage. See more
-details [here](https://docs.victoriametrics.com/Single-server-VictoriaMetrics.html#deduplication).
\ No newline at end of file
+details [here](https://docs.victoriametrics.com/Single-server-VictoriaMetrics.html#deduplication).
diff --git a/go.mod b/go.mod
index 087150c102..64d56a3d5b 100644
--- a/go.mod
+++ b/go.mod
@@ -3,7 +3,7 @@ module github.com/VictoriaMetrics/VictoriaMetrics
 go 1.17
 
 require (
-	cloud.google.com/go/storage v1.22.1
+	cloud.google.com/go/storage v1.23.0
 	github.com/VictoriaMetrics/fastcache v1.10.0
 
 	// Do not use the original github.com/valyala/fasthttp because of issues
@@ -11,7 +11,7 @@ require (
 	github.com/VictoriaMetrics/fasthttp v1.1.0
 	github.com/VictoriaMetrics/metrics v1.18.1
 	github.com/VictoriaMetrics/metricsql v0.44.0
-	github.com/aws/aws-sdk-go v1.44.37
+	github.com/aws/aws-sdk-go v1.44.43
 	github.com/cespare/xxhash/v2 v2.1.2
 	github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect
 
@@ -29,16 +29,16 @@ require (
 	github.com/oklog/ulid v1.3.1
 	github.com/prometheus/common v0.35.0 // indirect
 	github.com/prometheus/prometheus v1.8.2-0.20201119142752-3ad25a6dc3d9
-	github.com/urfave/cli/v2 v2.10.1
+	github.com/urfave/cli/v2 v2.10.3
 	github.com/valyala/fastjson v1.6.3
 	github.com/valyala/fastrand v1.1.0
 	github.com/valyala/fasttemplate v1.2.1
 	github.com/valyala/gozstd v1.17.0
 	github.com/valyala/quicktemplate v1.7.0
-	golang.org/x/net v0.0.0-20220617184016-355a448f1bc9
-	golang.org/x/oauth2 v0.0.0-20220608161450-d0670ef3b1eb
-	golang.org/x/sys v0.0.0-20220615213510-4f61da869c0c
-	google.golang.org/api v0.84.0
+	golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e
+	golang.org/x/oauth2 v0.0.0-20220622183110-fd043fe589d2
+	golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b
+	google.golang.org/api v0.85.0
 	gopkg.in/yaml.v2 v2.4.0
 )
 
@@ -76,7 +76,7 @@ require (
 	golang.org/x/text v0.3.7 // indirect
 	golang.org/x/xerrors v0.0.0-20220609144429-65e65417b02f // indirect
 	google.golang.org/appengine v1.6.7 // indirect
-	google.golang.org/genproto v0.0.0-20220617124728-180714bec0ad // indirect
+	google.golang.org/genproto v0.0.0-20220627200112-0a929928cb33 // indirect
 	google.golang.org/grpc v1.47.0 // indirect
 	google.golang.org/protobuf v1.28.0 // indirect
 )
diff --git a/go.sum b/go.sum
index 2432b16eef..7076687a49 100644
--- a/go.sum
+++ b/go.sum
@@ -59,8 +59,9 @@ cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0Zeo
 cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk=
 cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs=
 cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0=
-cloud.google.com/go/storage v1.22.1 h1:F6IlQJZrZM++apn9V5/VfS3gbTUYg98PS3EMQAzqtfg=
 cloud.google.com/go/storage v1.22.1/go.mod h1:S8N1cAStu7BOeFfE8KAQzmyyLkK8p/vmRq6kuBTW58Y=
+cloud.google.com/go/storage v1.23.0 h1:wWRIaDURQA8xxHguFCshYepGlrWIrbBnAmc7wfg07qY=
+cloud.google.com/go/storage v1.23.0/go.mod h1:vOEEDNFnciUMhBeT6hsJIn3ieU5cFRmzeLgDvXzfIXc=
 collectd.org v0.3.0/go.mod h1:A/8DzQBkF6abtvrT2j/AU/4tiBgJWYyh0y/oB/4MlWE=
 dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
 github.com/Azure/azure-sdk-for-go v48.2.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc=
@@ -144,8 +145,8 @@ github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQ
 github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
 github.com/aws/aws-sdk-go v1.34.28/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48=
 github.com/aws/aws-sdk-go v1.35.31/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro=
-github.com/aws/aws-sdk-go v1.44.37 h1:KvDxCX6dfJeEDC77U5GPGSP0ErecmNnhDHFxw+NIvlI=
-github.com/aws/aws-sdk-go v1.44.37/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX4oIKwKHZo=
+github.com/aws/aws-sdk-go v1.44.43 h1:gILXnQAOkfAV9dhdXOUlnVTGM3AiOQFqwQmJJ9R7rUE=
+github.com/aws/aws-sdk-go v1.44.43/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX4oIKwKHZo=
 github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g=
 github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
@@ -820,8 +821,8 @@ github.com/uber/jaeger-client-go v2.25.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMW
 github.com/uber/jaeger-lib v2.4.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U=
 github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
 github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
-github.com/urfave/cli/v2 v2.10.1 h1:34qJSQxqF/4fqJ7oiAV5WoXaTFlGG9QNM+qxpY3W3gs=
-github.com/urfave/cli/v2 v2.10.1/go.mod h1:MaQ2eKodtz1fFzu2U0jL+tVjoWmG134POMRjyXJK6+8=
+github.com/urfave/cli/v2 v2.10.3 h1:oi571Fxz5aHugfBAJd5nkwSk3fzATXtMlpxdLylSCMo=
+github.com/urfave/cli/v2 v2.10.3/go.mod h1:f8iq5LtQ/bLxafbdBSLPPNsgaW0l/2fYYEHhAyPlwvo=
 github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6KllzawFIhcdPw=
 github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
 github.com/valyala/fasthttp v1.30.0/go.mod h1:2rsYD01CKFrjjsvFxx75KlEUNpWNBY9JWD3K/7o2Cus=
@@ -998,8 +999,9 @@ golang.org/x/net v0.0.0-20220325170049-de3da57026de/go.mod h1:CfG3xpIq0wQ8r1q4Su
 golang.org/x/net v0.0.0-20220412020605-290c469a71a5/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk=
 golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk=
 golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
-golang.org/x/net v0.0.0-20220617184016-355a448f1bc9 h1:Yqz/iviulwKwAREEeUd3nbBFn0XuyJqkoft2IlrvOhc=
 golang.org/x/net v0.0.0-20220617184016-355a448f1bc9/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
+golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e h1:TsQ7F31D3bUCLeqPT0u+yjp1guoArKaNKmCr22PYgTQ=
+golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
 golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
 golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
 golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
@@ -1019,8 +1021,9 @@ golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8/go.mod h1:KelEdhl1UZF7XfJ
 golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc=
 golang.org/x/oauth2 v0.0.0-20220309155454-6242fa91716a/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc=
 golang.org/x/oauth2 v0.0.0-20220411215720-9780585627b5/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc=
-golang.org/x/oauth2 v0.0.0-20220608161450-d0670ef3b1eb h1:8tDJ3aechhddbdPAxpycgXHJRMLpk/Ab+aa4OgdN5/g=
 golang.org/x/oauth2 v0.0.0-20220608161450-d0670ef3b1eb/go.mod h1:jaDAt6Dkxork7LmZnYtzbRWj0W47D86a3TGe0YHBvmE=
+golang.org/x/oauth2 v0.0.0-20220622183110-fd043fe589d2 h1:+jnHzr9VPj32ykQVai5DNahi9+NSp7yYuCsl5eAQtL0=
+golang.org/x/oauth2 v0.0.0-20220622183110-fd043fe589d2/go.mod h1:jaDAt6Dkxork7LmZnYtzbRWj0W47D86a3TGe0YHBvmE=
 golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
@@ -1129,8 +1132,9 @@ golang.org/x/sys v0.0.0-20220502124256-b6088ccd6cba/go.mod h1:oPkhp1MJrh7nUepCBc
 golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/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-20220610221304-9f5ed59c137d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220615213510-4f61da869c0c h1:aFV+BgZ4svzjfabn8ERpuB4JI4N6/rdy1iusx77G3oU=
 golang.org/x/sys v0.0.0-20220615213510-4f61da869c0c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b h1:2n253B2r0pYSmEV+UNCQoPfU/FiaizQEK5Gu4Bq4JE8=
+golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
 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/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
@@ -1275,8 +1279,9 @@ google.golang.org/api v0.74.0/go.mod h1:ZpfMZOVRMywNyvJFeqL9HRWBgAuRfSjJFpe9QtRR
 google.golang.org/api v0.75.0/go.mod h1:pU9QmyHLnzlpar1Mjt4IbapUCy8J+6HD6GeELN69ljA=
 google.golang.org/api v0.78.0/go.mod h1:1Sg78yoMLOhlQTeF+ARBoytAcH1NNyyl390YMy6rKmw=
 google.golang.org/api v0.80.0/go.mod h1:xY3nI94gbvBrE0J6NHXhxOmW97HG7Khjkku6AFB3Hyg=
-google.golang.org/api v0.84.0 h1:NMB9J4cCxs9xEm+1Z9QiO3eFvn7EnQj3Eo3hN6ugVlg=
 google.golang.org/api v0.84.0/go.mod h1:NTsGnUFJMYROtiquksZHBWtHfeMC7iYthki7Eq3pa8o=
+google.golang.org/api v0.85.0 h1:8rJoHuRxx+vCmZtAO/3k1dRLvYNVyTJtZ5oaFZvhgvc=
+google.golang.org/api v0.85.0/go.mod h1:AqZf8Ep9uZ2pyTvgL+x0D3Zt0eoT9b5E8fmzfu6FO2g=
 google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
 google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
 google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
@@ -1367,8 +1372,9 @@ google.golang.org/genproto v0.0.0-20220518221133-4f43b3371335/go.mod h1:RAyBrSAP
 google.golang.org/genproto v0.0.0-20220523171625-347a074981d8/go.mod h1:RAyBrSAP7Fh3Nc84ghnVLDPuV51xc9agzmm4Ph6i0Q4=
 google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA=
 google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA=
-google.golang.org/genproto v0.0.0-20220617124728-180714bec0ad h1:kqrS+lhvaMHCxul6sKQvKJ8nAAhlVItmZV822hYFH/U=
 google.golang.org/genproto v0.0.0-20220617124728-180714bec0ad/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA=
+google.golang.org/genproto v0.0.0-20220627200112-0a929928cb33 h1:3L4edWcjDHPWGcMl1N0YH1NSoasyvfEcZCe2rUbxHfs=
+google.golang.org/genproto v0.0.0-20220627200112-0a929928cb33/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA=
 google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
 google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
 google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM=
diff --git a/lib/storage/metric_name.go b/lib/storage/metric_name.go
index 8e8f9418c1..4ae8624be1 100644
--- a/lib/storage/metric_name.go
+++ b/lib/storage/metric_name.go
@@ -3,6 +3,7 @@ package storage
 import (
 	"bytes"
 	"fmt"
+	"runtime"
 	"sort"
 	"strconv"
 	"strings"
@@ -390,6 +391,14 @@ func (mn *MetricName) Marshal(dst []byte) []byte {
 	return dst
 }
 
+// UnmarshalString unmarshals mn from s
+func (mn *MetricName) UnmarshalString(s string) error {
+	b := bytesutil.ToUnsafeBytes(s)
+	err := mn.Unmarshal(b)
+	runtime.KeepAlive(s)
+	return err
+}
+
 // Unmarshal unmarshals mn from src.
 func (mn *MetricName) Unmarshal(src []byte) error {
 	// Unmarshal MetricGroup.
diff --git a/lib/storage/storage.go b/lib/storage/storage.go
index 79faef092b..b461afe839 100644
--- a/lib/storage/storage.go
+++ b/lib/storage/storage.go
@@ -1081,8 +1081,10 @@ func nextRetentionDuration(retentionMsecs int64) time.Duration {
 	return time.Duration(deadline-t) * time.Millisecond
 }
 
-// SearchMetricNames returns metric names matching the given tfss on the given tr.
-func (s *Storage) SearchMetricNames(qt *querytracer.Tracer, tfss []*TagFilters, tr TimeRange, maxMetrics int, deadline uint64) ([]MetricName, error) {
+// SearchMetricNames returns marshaled metric names matching the given tfss on the given tr.
+//
+// The marshaled metric names must be unmarshaled via MetricName.UnmarshalString().
+func (s *Storage) SearchMetricNames(qt *querytracer.Tracer, tfss []*TagFilters, tr TimeRange, maxMetrics int, deadline uint64) ([]string, error) {
 	qt = qt.NewChild("search for matching metric names: filters=%s, timeRange=%s", tfss, &tr)
 	defer qt.Done()
 	tsids, err := s.searchTSIDs(qt, tfss, tr, maxMetrics, deadline)
@@ -1096,7 +1098,8 @@ func (s *Storage) SearchMetricNames(qt *querytracer.Tracer, tfss []*TagFilters,
 		return nil, err
 	}
 	idb := s.idb()
-	mns := make([]MetricName, 0, len(tsids))
+	metricNames := make([]string, 0, len(tsids))
+	metricNamesSeen := make(map[string]struct{}, len(tsids))
 	var metricName []byte
 	for i := range tsids {
 		if i&paceLimiterSlowIterationsMask == 0 {
@@ -1115,13 +1118,15 @@ func (s *Storage) SearchMetricNames(qt *querytracer.Tracer, tfss []*TagFilters,
 			}
 			return nil, fmt.Errorf("error when searching metricName for metricID=%d: %w", metricID, err)
 		}
-		mns = mns[:len(mns)+1]
-		mn := &mns[len(mns)-1]
-		if err = mn.Unmarshal(metricName); err != nil {
-			return nil, fmt.Errorf("cannot unmarshal metricName=%q: %w", metricName, err)
+		if _, ok := metricNamesSeen[string(metricName)]; ok {
+			// The given metric name was already seen; skip it
+			continue
 		}
+		metricNames = append(metricNames, string(metricName))
+		metricNamesSeen[metricNames[len(metricNames)-1]] = struct{}{}
 	}
-	return mns, nil
+	qt.Printf("loaded %d metric names", len(metricNames))
+	return metricNames, nil
 }
 
 // searchTSIDs returns sorted TSIDs for the given tfss and the given tr.
diff --git a/lib/storage/storage_test.go b/lib/storage/storage_test.go
index b316601223..0617882e0e 100644
--- a/lib/storage/storage_test.go
+++ b/lib/storage/storage_test.go
@@ -854,14 +854,21 @@ func testStorageRegisterMetricNames(s *Storage) error {
 	if err := tfs.Add([]byte("add_id"), []byte("0"), false, false); err != nil {
 		return fmt.Errorf("unexpected error in TagFilters.Add: %w", err)
 	}
-	mns, err := s.SearchMetricNames(nil, []*TagFilters{tfs}, tr, metricsPerAdd*addsCount*100+100, noDeadline)
+	metricNames, err := s.SearchMetricNames(nil, []*TagFilters{tfs}, tr, metricsPerAdd*addsCount*100+100, noDeadline)
 	if err != nil {
 		return fmt.Errorf("error in SearchMetricNames: %w", err)
 	}
-	if len(mns) < metricsPerAdd {
-		return fmt.Errorf("unexpected number of metricNames returned from SearchMetricNames; got %d; want at least %d", len(mns), int(metricsPerAdd))
+	if err != nil {
+		return fmt.Errorf("cannot unmarshal metric names: %w", err)
 	}
-	for i, mn := range mns {
+	if len(metricNames) < metricsPerAdd {
+		return fmt.Errorf("unexpected number of metricNames returned from SearchMetricNames; got %d; want at least %d", len(metricNames), int(metricsPerAdd))
+	}
+	var mn MetricName
+	for i, metricName := range metricNames {
+		if err := mn.UnmarshalString(metricName); err != nil {
+			return fmt.Errorf("cannot unmarshal metricName=%q: %w", metricName, err)
+		}
 		addID := mn.GetTagValue("add_id")
 		if string(addID) != "0" {
 			return fmt.Errorf("unexpected addID for metricName #%d; got %q; want %q", i, addID, "0")
diff --git a/vendor/cloud.google.com/go/storage/.release-please-manifest.json b/vendor/cloud.google.com/go/storage/.release-please-manifest.json
index a068903e03..110758e0c1 100644
--- a/vendor/cloud.google.com/go/storage/.release-please-manifest.json
+++ b/vendor/cloud.google.com/go/storage/.release-please-manifest.json
@@ -1,3 +1,3 @@
 {
-    "storage": "1.22.1"
+    "storage": "1.23.0"
 }
\ No newline at end of file
diff --git a/vendor/cloud.google.com/go/storage/CHANGES.md b/vendor/cloud.google.com/go/storage/CHANGES.md
index d88a1b2c8e..4a80b8ff4c 100644
--- a/vendor/cloud.google.com/go/storage/CHANGES.md
+++ b/vendor/cloud.google.com/go/storage/CHANGES.md
@@ -1,6 +1,19 @@
 # Changes
 
 
+## [1.23.0](https://github.com/googleapis/google-cloud-go/compare/storage/v1.22.1...storage/v1.23.0) (2022-06-23)
+
+
+### Features
+
+* **storage:** add support for OLM Prefix/Suffix ([#5929](https://github.com/googleapis/google-cloud-go/issues/5929)) ([ec21d10](https://github.com/googleapis/google-cloud-go/commit/ec21d10d6d1b01aa97a52560319775041707690d))
+* **storage:** support AbortIncompleteMultipartUpload LifecycleAction ([#5812](https://github.com/googleapis/google-cloud-go/issues/5812)) ([fdec929](https://github.com/googleapis/google-cloud-go/commit/fdec929b9da6e01dda0ab3c72544d44d6bd82bd4)), refs [#5795](https://github.com/googleapis/google-cloud-go/issues/5795)
+
+
+### Bug Fixes
+
+* **storage:** allow for  Age *int64 type and int64 type ([#6230](https://github.com/googleapis/google-cloud-go/issues/6230)) ([cc7acb8](https://github.com/googleapis/google-cloud-go/commit/cc7acb8bffb31828e9e96d4834a65f9728494473))
+
 ### [1.22.1](https://github.com/googleapis/google-cloud-go/compare/storage/v1.22.0...storage/v1.22.1) (2022-05-19)
 
 
diff --git a/vendor/cloud.google.com/go/storage/acl.go b/vendor/cloud.google.com/go/storage/acl.go
index 0c2374008b..13d79ee97e 100644
--- a/vendor/cloud.google.com/go/storage/acl.go
+++ b/vendor/cloud.google.com/go/storage/acl.go
@@ -67,6 +67,8 @@ type ProjectTeam struct {
 }
 
 // ACLHandle provides operations on an access control list for a Google Cloud Storage bucket or object.
+// ACLHandle on an object operates on the latest generation of that object by default.
+// Selecting a specific generation of an object is not currently supported by the client.
 type ACLHandle struct {
 	c           *Client
 	bucket      string
diff --git a/vendor/cloud.google.com/go/storage/bucket.go b/vendor/cloud.google.com/go/storage/bucket.go
index 101f695b5a..2148e68965 100644
--- a/vendor/cloud.google.com/go/storage/bucket.go
+++ b/vendor/cloud.google.com/go/storage/bucket.go
@@ -645,6 +645,13 @@ const (
 	// SetStorageClassAction changes the storage class of live and/or archived
 	// objects.
 	SetStorageClassAction = "SetStorageClass"
+
+	// AbortIncompleteMPUAction is a lifecycle action that aborts an incomplete
+	// multipart upload when the multipart upload meets the conditions specified
+	// in the lifecycle rule. The AgeInDays condition is the only allowed
+	// condition for this action. AgeInDays is measured from the time the
+	// multipart upload was created.
+	AbortIncompleteMPUAction = "AbortIncompleteMultipartUpload"
 )
 
 // LifecycleRule is a lifecycle configuration rule.
@@ -665,9 +672,8 @@ type LifecycleRule struct {
 type LifecycleAction struct {
 	// Type is the type of action to take on matching objects.
 	//
-	// Acceptable values are "Delete" to delete matching objects and
-	// "SetStorageClass" to set the storage class defined in StorageClass on
-	// matching objects.
+	// Acceptable values are storage.DeleteAction, storage.SetStorageClassAction,
+	// and storage.AbortIncompleteMPUAction.
 	Type string
 
 	// StorageClass is the storage class to set on matching objects if the Action
@@ -719,12 +725,20 @@ type LifecycleCondition struct {
 	// Liveness specifies the object's liveness. Relevant only for versioned objects
 	Liveness Liveness
 
+	// MatchesPrefix is the condition matching an object if any of the
+	// matches_prefix strings are an exact prefix of the object's name.
+	MatchesPrefix []string
+
 	// MatchesStorageClasses is the condition matching the object's storage
 	// class.
 	//
 	// Values include "STANDARD", "NEARLINE", "COLDLINE" and "ARCHIVE".
 	MatchesStorageClasses []string
 
+	// MatchesSuffix is the condition matching an object if any of the
+	// matches_suffix strings are an exact suffix of the object's name.
+	MatchesSuffix []string
+
 	// NoncurrentTimeBefore is the noncurrent timestamp of the object. This
 	// condition is satisfied when an object's noncurrent timestamp is before
 	// midnight of the specified date in UTC.
@@ -1489,6 +1503,19 @@ func toCORSFromProto(rc []*storagepb.Bucket_Cors) []CORS {
 	return out
 }
 
+// Used to handle breaking change in Autogen Storage client OLM Age field
+// from int64 to *int64 gracefully in the manual client
+// TODO(#6240): Method should be removed once breaking change is made and introduced to this client
+func setAgeCondition(age int64, ageField interface{}) {
+	c := reflect.ValueOf(ageField).Elem()
+	switch c.Kind() {
+	case reflect.Int64:
+		c.SetInt(age)
+	case reflect.Ptr:
+		c.Set(reflect.ValueOf(&age))
+	}
+}
+
 func toRawLifecycle(l Lifecycle) *raw.BucketLifecycle {
 	var rl raw.BucketLifecycle
 	if len(l.Rules) == 0 {
@@ -1501,14 +1528,17 @@ func toRawLifecycle(l Lifecycle) *raw.BucketLifecycle {
 				StorageClass: r.Action.StorageClass,
 			},
 			Condition: &raw.BucketLifecycleRuleCondition{
-				Age:                     r.Condition.AgeInDays,
 				DaysSinceCustomTime:     r.Condition.DaysSinceCustomTime,
 				DaysSinceNoncurrentTime: r.Condition.DaysSinceNoncurrentTime,
+				MatchesPrefix:           r.Condition.MatchesPrefix,
 				MatchesStorageClass:     r.Condition.MatchesStorageClasses,
+				MatchesSuffix:           r.Condition.MatchesSuffix,
 				NumNewerVersions:        r.Condition.NumNewerVersions,
 			},
 		}
 
+		setAgeCondition(r.Condition.AgeInDays, &rr.Condition.Age)
+
 		switch r.Condition.Liveness {
 		case LiveAndArchived:
 			rr.Condition.IsLive = nil
@@ -1549,7 +1579,9 @@ func toProtoLifecycle(l Lifecycle) *storagepb.Bucket_Lifecycle {
 				AgeDays:                 proto.Int32(int32(r.Condition.AgeInDays)),
 				DaysSinceCustomTime:     proto.Int32(int32(r.Condition.DaysSinceCustomTime)),
 				DaysSinceNoncurrentTime: proto.Int32(int32(r.Condition.DaysSinceNoncurrentTime)),
+				MatchesPrefix:           r.Condition.MatchesPrefix,
 				MatchesStorageClass:     r.Condition.MatchesStorageClasses,
+				MatchesSuffix:           r.Condition.MatchesSuffix,
 				NumNewerVersions:        proto.Int32(int32(r.Condition.NumNewerVersions)),
 			},
 		}
@@ -1577,6 +1609,21 @@ func toProtoLifecycle(l Lifecycle) *storagepb.Bucket_Lifecycle {
 	return &rl
 }
 
+// Used to handle breaking change in Autogen Storage client OLM Age field
+// from int64 to *int64 gracefully in the manual client
+// TODO(#6240): Method should be removed once breaking change is made and introduced to this client
+func getAgeCondition(ageField interface{}) int64 {
+	v := reflect.ValueOf(ageField)
+	if v.Kind() == reflect.Int64 {
+		return v.Interface().(int64)
+	} else if v.Kind() == reflect.Ptr {
+		if val, ok := v.Interface().(*int64); ok {
+			return *val
+		}
+	}
+	return 0
+}
+
 func toLifecycle(rl *raw.BucketLifecycle) Lifecycle {
 	var l Lifecycle
 	if rl == nil {
@@ -1589,13 +1636,15 @@ func toLifecycle(rl *raw.BucketLifecycle) Lifecycle {
 				StorageClass: rr.Action.StorageClass,
 			},
 			Condition: LifecycleCondition{
-				AgeInDays:               rr.Condition.Age,
 				DaysSinceCustomTime:     rr.Condition.DaysSinceCustomTime,
 				DaysSinceNoncurrentTime: rr.Condition.DaysSinceNoncurrentTime,
+				MatchesPrefix:           rr.Condition.MatchesPrefix,
 				MatchesStorageClasses:   rr.Condition.MatchesStorageClass,
+				MatchesSuffix:           rr.Condition.MatchesSuffix,
 				NumNewerVersions:        rr.Condition.NumNewerVersions,
 			},
 		}
+		r.Condition.AgeInDays = getAgeCondition(rr.Condition.Age)
 
 		if rr.Condition.IsLive == nil {
 			r.Condition.Liveness = LiveAndArchived
@@ -1634,7 +1683,9 @@ func toLifecycleFromProto(rl *storagepb.Bucket_Lifecycle) Lifecycle {
 				AgeInDays:               int64(rr.GetCondition().GetAgeDays()),
 				DaysSinceCustomTime:     int64(rr.GetCondition().GetDaysSinceCustomTime()),
 				DaysSinceNoncurrentTime: int64(rr.GetCondition().GetDaysSinceNoncurrentTime()),
+				MatchesPrefix:           rr.GetCondition().GetMatchesPrefix(),
 				MatchesStorageClasses:   rr.GetCondition().GetMatchesStorageClass(),
+				MatchesSuffix:           rr.GetCondition().GetMatchesSuffix(),
 				NumNewerVersions:        int64(rr.GetCondition().GetNumNewerVersions()),
 			},
 		}
@@ -1708,7 +1759,7 @@ func (b *BucketLogging) toProtoBucketLogging() *storagepb.Bucket_Logging {
 		return nil
 	}
 	return &storagepb.Bucket_Logging{
-		LogBucket:       b.LogBucket,
+		LogBucket:       bucketResourceName(globalProjectAlias, b.LogBucket),
 		LogObjectPrefix: b.LogObjectPrefix,
 	}
 }
@@ -1727,8 +1778,9 @@ func toBucketLoggingFromProto(b *storagepb.Bucket_Logging) *BucketLogging {
 	if b == nil {
 		return nil
 	}
+	lb := parseBucketName(b.GetLogBucket())
 	return &BucketLogging{
-		LogBucket:       b.GetLogBucket(),
+		LogBucket:       lb,
 		LogObjectPrefix: b.GetLogObjectPrefix(),
 	}
 }
diff --git a/vendor/cloud.google.com/go/storage/client.go b/vendor/cloud.google.com/go/storage/client.go
index 87ee41927d..f26a52a765 100644
--- a/vendor/cloud.google.com/go/storage/client.go
+++ b/vendor/cloud.google.com/go/storage/client.go
@@ -16,6 +16,8 @@ package storage
 
 import (
 	"context"
+	"io"
+	"time"
 
 	gax "github.com/googleapis/gax-go/v2"
 	"google.golang.org/api/option"
@@ -56,9 +58,9 @@ type storageClient interface {
 
 	// Object metadata methods.
 
-	DeleteObject(ctx context.Context, bucket, object string, conds *Conditions, opts ...storageOption) error
-	GetObject(ctx context.Context, bucket, object string, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error)
-	UpdateObject(ctx context.Context, bucket, object string, uattrs *ObjectAttrsToUpdate, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error)
+	DeleteObject(ctx context.Context, bucket, object string, gen int64, conds *Conditions, opts ...storageOption) error
+	GetObject(ctx context.Context, bucket, object string, gen int64, encryptionKey []byte, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error)
+	UpdateObject(ctx context.Context, bucket, object string, uattrs *ObjectAttrsToUpdate, gen int64, encryptionKey []byte, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error)
 
 	// Default Object ACL methods.
 
@@ -83,8 +85,8 @@ type storageClient interface {
 	ComposeObject(ctx context.Context, req *composeObjectRequest, opts ...storageOption) (*ObjectAttrs, error)
 	RewriteObject(ctx context.Context, req *rewriteObjectRequest, opts ...storageOption) (*rewriteObjectResponse, error)
 
-	OpenReader(ctx context.Context, r *Reader, opts ...storageOption) error
-	OpenWriter(ctx context.Context, w *Writer, opts ...storageOption) error
+	NewRangeReader(ctx context.Context, params *newRangeReaderParams, opts ...storageOption) (*Reader, error)
+	OpenWriter(params *openWriterParams, opts ...storageOption) (*io.PipeWriter, error)
 
 	// IAM methods.
 
@@ -99,6 +101,11 @@ type storageClient interface {
 	UpdateHMACKey(ctx context.Context, desc *hmacKeyDesc, attrs *HMACKeyAttrsToUpdate, opts ...storageOption) (*HMACKey, error)
 	CreateHMACKey(ctx context.Context, desc *hmacKeyDesc, opts ...storageOption) (*HMACKey, error)
 	DeleteHMACKey(ctx context.Context, desc *hmacKeyDesc, opts ...storageOption) error
+
+	// Notification methods.
+	ListNotifications(ctx context.Context, bucket string, opts ...storageOption) (map[string]*Notification, error)
+	CreateNotification(ctx context.Context, bucket string, n *Notification, opts ...storageOption) (*Notification, error)
+	DeleteNotification(ctx context.Context, bucket string, id string, opts ...storageOption) error
 }
 
 // settings contains transport-agnostic configuration for API calls made via
@@ -211,6 +218,64 @@ type userProjectOption struct {
 
 func (o *userProjectOption) Apply(s *settings) { s.userProject = o.project }
 
+type openWriterParams struct {
+	// Writer configuration
+
+	// ctx is the context used by the writer routine to make all network calls
+	// and to manage the writer routine - see `Writer.ctx`.
+	// Required.
+	ctx context.Context
+	// chunkSize - see `Writer.ChunkSize`.
+	// Optional.
+	chunkSize int
+	// chunkRetryDeadline - see `Writer.ChunkRetryDeadline`.
+	// Optional.
+	chunkRetryDeadline time.Duration
+
+	// Object/request properties
+
+	// bucket - see `Writer.o.bucket`.
+	// Required.
+	bucket string
+	// attrs - see `Writer.ObjectAttrs`.
+	// Required.
+	attrs *ObjectAttrs
+	// conds - see `Writer.o.conds`.
+	// Optional.
+	conds *Conditions
+	// encryptionKey - see `Writer.o.encryptionKey`
+	// Optional.
+	encryptionKey []byte
+	// sendCRC32C - see `Writer.SendCRC32C`.
+	// Optional.
+	sendCRC32C bool
+
+	// Writer callbacks
+
+	// donec - see `Writer.donec`.
+	// Required.
+	donec chan struct{}
+	// setError callback for reporting errors - see `Writer.error`.
+	// Required.
+	setError func(error)
+	// progress callback for reporting upload progress - see `Writer.progress`.
+	// Required.
+	progress func(int64)
+	// setObj callback for reporting the resulting object - see `Writer.obj`.
+	// Required.
+	setObj func(*ObjectAttrs)
+}
+
+type newRangeReaderParams struct {
+	bucket        string
+	conds         *Conditions
+	encryptionKey []byte
+	gen           int64
+	length        int64
+	object        string
+	offset        int64
+}
+
 type composeObjectRequest struct {
 	dstBucket     string
 	dstObject     string
diff --git a/vendor/cloud.google.com/go/storage/grpc_client.go b/vendor/cloud.google.com/go/storage/grpc_client.go
index db914d61f3..cc246cf724 100644
--- a/vendor/cloud.google.com/go/storage/grpc_client.go
+++ b/vendor/cloud.google.com/go/storage/grpc_client.go
@@ -16,8 +16,11 @@ package storage
 
 import (
 	"context"
+	"fmt"
+	"io"
 	"os"
 
+	"cloud.google.com/go/internal/trace"
 	gapic "cloud.google.com/go/storage/internal/apiv2"
 	"google.golang.org/api/iterator"
 	"google.golang.org/api/option"
@@ -27,6 +30,7 @@ import (
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/metadata"
 	"google.golang.org/grpc/status"
+	"google.golang.org/protobuf/proto"
 	fieldmaskpb "google.golang.org/protobuf/types/known/fieldmaskpb"
 )
 
@@ -36,7 +40,7 @@ const (
 	// connection pool may be necessary for jobs that require
 	// high throughput and/or leverage many concurrent streams.
 	//
-	// This is an experimental API and not intended for public use.
+	// This is only used for the gRPC client.
 	defaultConnPoolSize = 4
 
 	// globalProjectAlias is the project ID alias used for global buckets.
@@ -47,8 +51,6 @@ const (
 
 // defaultGRPCOptions returns a set of the default client options
 // for gRPC client initialization.
-//
-// This is an experimental API and not intended for public use.
 func defaultGRPCOptions() []option.ClientOption {
 	defaults := []option.ClientOption{
 		option.WithGRPCConnectionPool(defaultConnPoolSize),
@@ -80,8 +82,6 @@ func defaultGRPCOptions() []option.ClientOption {
 
 // grpcStorageClient is the gRPC API implementation of the transport-agnostic
 // storageClient interface.
-//
-// This is an experimental API and not intended for public use.
 type grpcStorageClient struct {
 	raw      *gapic.Client
 	settings *settings
@@ -89,8 +89,6 @@ type grpcStorageClient struct {
 
 // newGRPCStorageClient initializes a new storageClient that uses the gRPC
 // Storage API.
-//
-// This is an experimental API and not intended for public use.
 func newGRPCStorageClient(ctx context.Context, opts ...storageOption) (storageClient, error) {
 	s := initSettings(opts...)
 	s.clientOption = append(defaultGRPCOptions(), s.clientOption...)
@@ -404,14 +402,119 @@ func (c *grpcStorageClient) ListObjects(ctx context.Context, bucket string, q *Q
 
 // Object metadata methods.
 
-func (c *grpcStorageClient) DeleteObject(ctx context.Context, bucket, object string, conds *Conditions, opts ...storageOption) error {
-	return errMethodNotSupported
+func (c *grpcStorageClient) DeleteObject(ctx context.Context, bucket, object string, gen int64, conds *Conditions, opts ...storageOption) error {
+	s := callSettings(c.settings, opts...)
+	req := &storagepb.DeleteObjectRequest{
+		Bucket: bucketResourceName(globalProjectAlias, bucket),
+		Object: object,
+	}
+	if err := applyCondsProto("grpcStorageClient.DeleteObject", gen, conds, req); err != nil {
+		return err
+	}
+	if s.userProject != "" {
+		ctx = setUserProjectMetadata(ctx, s.userProject)
+	}
+	err := run(ctx, func() error {
+		return c.raw.DeleteObject(ctx, req, s.gax...)
+	}, s.retry, s.idempotent, setRetryHeaderGRPC(ctx))
+	if s, ok := status.FromError(err); ok && s.Code() == codes.NotFound {
+		return ErrObjectNotExist
+	}
+	return err
 }
-func (c *grpcStorageClient) GetObject(ctx context.Context, bucket, object string, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
-	return nil, errMethodNotSupported
+
+func (c *grpcStorageClient) GetObject(ctx context.Context, bucket, object string, gen int64, encryptionKey []byte, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
+	s := callSettings(c.settings, opts...)
+	req := &storagepb.GetObjectRequest{
+		Bucket: bucketResourceName(globalProjectAlias, bucket),
+		Object: object,
+	}
+	if err := applyCondsProto("grpcStorageClient.GetObject", gen, conds, req); err != nil {
+		return nil, err
+	}
+	if s.userProject != "" {
+		ctx = setUserProjectMetadata(ctx, s.userProject)
+	}
+	if encryptionKey != nil {
+		req.CommonObjectRequestParams = toProtoCommonObjectRequestParams(encryptionKey)
+	}
+
+	var attrs *ObjectAttrs
+	err := run(ctx, func() error {
+		res, err := c.raw.GetObject(ctx, req, s.gax...)
+		attrs = newObjectFromProto(res)
+
+		return err
+	}, s.retry, s.idempotent, setRetryHeaderGRPC(ctx))
+
+	if s, ok := status.FromError(err); ok && s.Code() == codes.NotFound {
+		return nil, ErrObjectNotExist
+	}
+
+	return attrs, err
 }
-func (c *grpcStorageClient) UpdateObject(ctx context.Context, bucket, object string, uattrs *ObjectAttrsToUpdate, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
-	return nil, errMethodNotSupported
+
+func (c *grpcStorageClient) UpdateObject(ctx context.Context, bucket, object string, uattrs *ObjectAttrsToUpdate, gen int64, encryptionKey []byte, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
+	s := callSettings(c.settings, opts...)
+	o := uattrs.toProtoObject(bucketResourceName(globalProjectAlias, bucket), object)
+	req := &storagepb.UpdateObjectRequest{
+		Object: o,
+	}
+	if err := applyCondsProto("grpcStorageClient.UpdateObject", gen, conds, req); err != nil {
+		return nil, err
+	}
+	if s.userProject != "" {
+		ctx = setUserProjectMetadata(ctx, s.userProject)
+	}
+	if encryptionKey != nil {
+		req.CommonObjectRequestParams = toProtoCommonObjectRequestParams(encryptionKey)
+	}
+
+	var paths []string
+	fieldMask := &fieldmaskpb.FieldMask{
+		Paths: paths,
+	}
+	if uattrs.EventBasedHold != nil {
+		fieldMask.Paths = append(fieldMask.Paths, "event_based_hold")
+	}
+	if uattrs.TemporaryHold != nil {
+		fieldMask.Paths = append(fieldMask.Paths, "temporary_hold")
+	}
+	if uattrs.ContentType != nil {
+		fieldMask.Paths = append(fieldMask.Paths, "content_type")
+	}
+	if uattrs.ContentLanguage != nil {
+		fieldMask.Paths = append(fieldMask.Paths, "content_language")
+	}
+	if uattrs.ContentEncoding != nil {
+		fieldMask.Paths = append(fieldMask.Paths, "content_encoding")
+	}
+	if uattrs.ContentDisposition != nil {
+		fieldMask.Paths = append(fieldMask.Paths, "content_disposition")
+	}
+	if uattrs.CacheControl != nil {
+		fieldMask.Paths = append(fieldMask.Paths, "cache_control")
+	}
+	if !uattrs.CustomTime.IsZero() {
+		fieldMask.Paths = append(fieldMask.Paths, "custom_time")
+	}
+
+	// TODO(cathyo): Handle ACL and PredefinedACL. Pending b/233617896.
+	// TODO(cathyo): Handle metadata. Pending b/230510191.
+
+	req.UpdateMask = fieldMask
+
+	var attrs *ObjectAttrs
+	err := run(ctx, func() error {
+		res, err := c.raw.UpdateObject(ctx, req, s.gax...)
+		attrs = newObjectFromProto(res)
+		return err
+	}, s.retry, s.idempotent, setRetryHeaderGRPC(ctx))
+	if e, ok := status.FromError(err); ok && e.Code() == codes.NotFound {
+		return nil, ErrObjectNotExist
+	}
+
+	return attrs, err
 }
 
 // Default Object ACL methods.
@@ -503,9 +606,17 @@ func (c *grpcStorageClient) UpdateBucketACL(ctx context.Context, bucket string,
 func (c *grpcStorageClient) DeleteObjectACL(ctx context.Context, bucket, object string, entity ACLEntity, opts ...storageOption) error {
 	return errMethodNotSupported
 }
+
+// ListObjectACLs retrieves object ACL entries. By default, it operates on the latest generation of this object.
+// Selecting a specific generation of this object is not currently supported by the client.
 func (c *grpcStorageClient) ListObjectACLs(ctx context.Context, bucket, object string, opts ...storageOption) ([]ACLRule, error) {
-	return nil, errMethodNotSupported
+	o, err := c.GetObject(ctx, bucket, object, defaultGen, nil, nil, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return o.ACL, nil
 }
+
 func (c *grpcStorageClient) UpdateObjectACL(ctx context.Context, bucket, object string, entity ACLEntity, role ACLRole, opts ...storageOption) (*ACLRule, error) {
 	return nil, errMethodNotSupported
 }
@@ -519,11 +630,200 @@ func (c *grpcStorageClient) RewriteObject(ctx context.Context, req *rewriteObjec
 	return nil, errMethodNotSupported
 }
 
-func (c *grpcStorageClient) OpenReader(ctx context.Context, r *Reader, opts ...storageOption) error {
-	return errMethodNotSupported
+func (c *grpcStorageClient) NewRangeReader(ctx context.Context, params *newRangeReaderParams, opts ...storageOption) (r *Reader, err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.grpcStorageClient.NewRangeReader")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	if params.conds != nil {
+		if err := params.conds.validate("grpcStorageClient.NewRangeReader"); err != nil {
+			return nil, err
+		}
+	}
+
+	s := callSettings(c.settings, opts...)
+
+	// A negative length means "read to the end of the object", but the
+	// read_limit field it corresponds to uses zero to mean the same thing. Thus
+	// we coerce the length to 0 to read to the end of the object.
+	if params.length < 0 {
+		params.length = 0
+	}
+
+	b := bucketResourceName(globalProjectAlias, params.bucket)
+	// TODO(noahdietz): Use encryptionKey to set relevant request fields.
+	req := &storagepb.ReadObjectRequest{
+		Bucket: b,
+		Object: params.object,
+	}
+	// The default is a negative value, which means latest.
+	if params.gen >= 0 {
+		req.Generation = params.gen
+	}
+
+	// Define a function that initiates a Read with offset and length, assuming
+	// we have already read seen bytes.
+	reopen := func(seen int64) (*readStreamResponse, context.CancelFunc, error) {
+		// If the context has already expired, return immediately without making
+		// we call.
+		if err := ctx.Err(); err != nil {
+			return nil, nil, err
+		}
+
+		cc, cancel := context.WithCancel(ctx)
+
+		start := params.offset + seen
+		// Only set a ReadLimit if length is greater than zero, because zero
+		// means read it all.
+		if params.length > 0 {
+			req.ReadLimit = params.length - seen
+		}
+		req.ReadOffset = start
+
+		if err := applyCondsProto("gRPCReader.reopen", params.gen, params.conds, req); err != nil {
+			cancel()
+			return nil, nil, err
+		}
+
+		var stream storagepb.Storage_ReadObjectClient
+		var msg *storagepb.ReadObjectResponse
+		var err error
+
+		err = run(cc, func() error {
+			stream, err = c.raw.ReadObject(cc, req, s.gax...)
+			if err != nil {
+				return err
+			}
+
+			msg, err = stream.Recv()
+
+			return err
+		}, s.retry, s.idempotent, setRetryHeaderGRPC(ctx))
+		if err != nil {
+			// Close the stream context we just created to ensure we don't leak
+			// resources.
+			cancel()
+			return nil, nil, err
+		}
+
+		return &readStreamResponse{stream, msg}, cancel, nil
+	}
+
+	res, cancel, err := reopen(0)
+	if err != nil {
+		return nil, err
+	}
+
+	// The first message was Recv'd on stream open, use it to populate the
+	// object metadata.
+	msg := res.response
+	obj := msg.GetMetadata()
+	// This is the size of the entire object, even if only a range was requested.
+	size := obj.GetSize()
+
+	r = &Reader{
+		Attrs: ReaderObjectAttrs{
+			Size:            size,
+			ContentType:     obj.GetContentType(),
+			ContentEncoding: obj.GetContentEncoding(),
+			CacheControl:    obj.GetCacheControl(),
+			LastModified:    obj.GetUpdateTime().AsTime(),
+			Metageneration:  obj.GetMetageneration(),
+			Generation:      obj.GetGeneration(),
+		},
+		reader: &gRPCReader{
+			stream: res.stream,
+			reopen: reopen,
+			cancel: cancel,
+			size:   size,
+			// Store the content from the first Recv in the
+			// client buffer for reading later.
+			leftovers: msg.GetChecksummedData().GetContent(),
+		},
+	}
+
+	cr := msg.GetContentRange()
+	if cr != nil {
+		r.Attrs.StartOffset = cr.GetStart()
+		r.remain = cr.GetEnd() - cr.GetStart() + 1
+	} else {
+		r.remain = size
+	}
+
+	// Only support checksums when reading an entire object, not a range.
+	if checksums := msg.GetObjectChecksums(); checksums != nil && checksums.Crc32C != nil && params.offset == 0 && params.length == 0 {
+		r.wantCRC = checksums.GetCrc32C()
+		r.checkCRC = true
+	}
+
+	return r, nil
 }
-func (c *grpcStorageClient) OpenWriter(ctx context.Context, w *Writer, opts ...storageOption) error {
-	return errMethodNotSupported
+
+func (c *grpcStorageClient) OpenWriter(params *openWriterParams, opts ...storageOption) (*io.PipeWriter, error) {
+	var offset int64
+	errorf := params.setError
+	progress := params.progress
+	setObj := params.setObj
+
+	pr, pw := io.Pipe()
+	gw := newGRPCWriter(c, params, pr)
+
+	// This function reads the data sent to the pipe and sends sets of messages
+	// on the gRPC client-stream as the buffer is filled.
+	go func() {
+		defer close(params.donec)
+
+		// Loop until there is an error or the Object has been finalized.
+		for {
+			// Note: This blocks until either the buffer is full or EOF is read.
+			recvd, doneReading, err := gw.read()
+			if err != nil {
+				err = checkCanceled(err)
+				errorf(err)
+				pr.CloseWithError(err)
+				return
+			}
+
+			// TODO(noahdietz): Send encryption key via CommonObjectRequestParams.
+
+			// The chunk buffer is full, but there is no end in sight. This
+			// means that a resumable upload will need to be used to send
+			// multiple chunks, until we are done reading data. Start a
+			// resumable upload if it has not already been started.
+			// Otherwise, all data will be sent over a single gRPC stream.
+			if !doneReading && gw.upid == "" {
+				err = gw.startResumableUpload()
+				if err != nil {
+					err = checkCanceled(err)
+					errorf(err)
+					pr.CloseWithError(err)
+					return
+				}
+			}
+
+			o, off, finalized, err := gw.uploadBuffer(recvd, offset, doneReading)
+			if err != nil {
+				err = checkCanceled(err)
+				errorf(err)
+				pr.CloseWithError(err)
+				return
+			}
+			// At this point, the current buffer has been uploaded. Capture the
+			// committed offset here in case the upload was not finalized and
+			// another chunk is to be uploaded.
+			offset = off
+			progress(offset)
+
+			// When we are done reading data and the chunk has been finalized,
+			// we are done.
+			if doneReading && finalized {
+				// Build Object from server's response.
+				setObj(newObjectFromProto(o))
+				return
+			}
+		}
+	}()
+
+	return pw, nil
 }
 
 // IAM methods.
@@ -599,6 +899,75 @@ func (c *grpcStorageClient) DeleteHMACKey(ctx context.Context, desc *hmacKeyDesc
 	return errMethodNotSupported
 }
 
+// Notification methods.
+
+func (c *grpcStorageClient) ListNotifications(ctx context.Context, bucket string, opts ...storageOption) (n map[string]*Notification, err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.grpcStorageClient.ListNotifications")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	s := callSettings(c.settings, opts...)
+	if s.userProject != "" {
+		ctx = setUserProjectMetadata(ctx, s.userProject)
+	}
+	req := &storagepb.ListNotificationsRequest{
+		Parent: bucketResourceName(globalProjectAlias, bucket),
+	}
+	var notifications []*storagepb.Notification
+	err = run(ctx, func() error {
+		gitr := c.raw.ListNotifications(ctx, req, s.gax...)
+		for {
+			// PageSize is not set and fallbacks to the API default pageSize of 100.
+			items, nextPageToken, err := gitr.InternalFetch(int(req.GetPageSize()), req.GetPageToken())
+			if err != nil {
+				return err
+			}
+			notifications = append(notifications, items...)
+			// If there are no more results, nextPageToken is empty and err is nil.
+			if nextPageToken == "" {
+				return err
+			}
+			req.PageToken = nextPageToken
+		}
+	}, s.retry, s.idempotent, setRetryHeaderGRPC(ctx))
+	if err != nil {
+		return nil, err
+	}
+
+	return notificationsToMapFromProto(notifications), nil
+}
+
+func (c *grpcStorageClient) CreateNotification(ctx context.Context, bucket string, n *Notification, opts ...storageOption) (ret *Notification, err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.grpcStorageClient.CreateNotification")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	s := callSettings(c.settings, opts...)
+	req := &storagepb.CreateNotificationRequest{
+		Parent:       bucketResourceName(globalProjectAlias, bucket),
+		Notification: toProtoNotification(n),
+	}
+	var pbn *storagepb.Notification
+	err = run(ctx, func() error {
+		var err error
+		pbn, err = c.raw.CreateNotification(ctx, req, s.gax...)
+		return err
+	}, s.retry, s.idempotent, setRetryHeaderGRPC(ctx))
+	if err != nil {
+		return nil, err
+	}
+	return toNotificationFromProto(pbn), err
+}
+
+func (c *grpcStorageClient) DeleteNotification(ctx context.Context, bucket string, id string, opts ...storageOption) (err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.grpcStorageClient.DeleteNotification")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	s := callSettings(c.settings, opts...)
+	req := &storagepb.DeleteNotificationRequest{Name: id}
+	return run(ctx, func() error {
+		return c.raw.DeleteNotification(ctx, req, s.gax...)
+	}, s.retry, s.idempotent, setRetryHeaderGRPC(ctx))
+}
+
 // setUserProjectMetadata appends a project ID to the outgoing Context metadata
 // via the x-goog-user-project system parameter defined at
 // https://cloud.google.com/apis/docs/system-parameters. This is only for
@@ -607,3 +976,396 @@ func (c *grpcStorageClient) DeleteHMACKey(ctx context.Context, desc *hmacKeyDesc
 func setUserProjectMetadata(ctx context.Context, project string) context.Context {
 	return metadata.AppendToOutgoingContext(ctx, "x-goog-user-project", project)
 }
+
+type readStreamResponse struct {
+	stream   storagepb.Storage_ReadObjectClient
+	response *storagepb.ReadObjectResponse
+}
+
+type gRPCReader struct {
+	seen, size int64
+	stream     storagepb.Storage_ReadObjectClient
+	reopen     func(seen int64) (*readStreamResponse, context.CancelFunc, error)
+	leftovers  []byte
+	cancel     context.CancelFunc
+}
+
+// Read reads bytes into the user's buffer from an open gRPC stream.
+func (r *gRPCReader) Read(p []byte) (int, error) {
+	// No stream to read from, either never initiliazed or Close was called.
+	// Note: There is a potential concurrency issue if multiple routines are
+	// using the same reader. One encounters an error and the stream is closed
+	// and then reopened while the other routine attempts to read from it.
+	if r.stream == nil {
+		return 0, fmt.Errorf("reader has been closed")
+	}
+
+	// The entire object has been read by this reader, return EOF.
+	if r.size != 0 && r.size == r.seen {
+		return 0, io.EOF
+	}
+
+	var n int
+	// Read leftovers and return what was available to conform to the Reader
+	// interface: https://pkg.go.dev/io#Reader.
+	if len(r.leftovers) > 0 {
+		n = copy(p, r.leftovers)
+		r.seen += int64(n)
+		r.leftovers = r.leftovers[n:]
+		return n, nil
+	}
+
+	// Attempt to Recv the next message on the stream.
+	msg, err := r.recv()
+	if err != nil {
+		return 0, err
+	}
+
+	// TODO: Determine if we need to capture incremental CRC32C for this
+	// chunk. The Object CRC32C checksum is captured when directed to read
+	// the entire Object. If directed to read a range, we may need to
+	// calculate the range's checksum for verification if the checksum is
+	// present in the response here.
+	// TODO: Figure out if we need to support decompressive transcoding
+	// https://cloud.google.com/storage/docs/transcoding.
+	content := msg.GetChecksummedData().GetContent()
+	n = copy(p[n:], content)
+	leftover := len(content) - n
+	if leftover > 0 {
+		// Wasn't able to copy all of the data in the message, store for
+		// future Read calls.
+		r.leftovers = content[n:]
+	}
+	r.seen += int64(n)
+
+	return n, nil
+}
+
+// Close cancels the read stream's context in order for it to be closed and
+// collected.
+func (r *gRPCReader) Close() error {
+	if r.cancel != nil {
+		r.cancel()
+	}
+	r.stream = nil
+	return nil
+}
+
+// recv attempts to Recv the next message on the stream. In the event
+// that a retryable error is encountered, the stream will be closed, reopened,
+// and Recv again. This will attempt to Recv until one of the following is true:
+//
+// * Recv is successful
+// * A non-retryable error is encountered
+// * The Reader's context is canceled
+//
+// The last error received is the one that is returned, which could be from
+// an attempt to reopen the stream.
+func (r *gRPCReader) recv() (*storagepb.ReadObjectResponse, error) {
+	msg, err := r.stream.Recv()
+	if err != nil && shouldRetry(err) {
+		// This will "close" the existing stream and immediately attempt to
+		// reopen the stream, but will backoff if further attempts are necessary.
+		// Reopening the stream Recvs the first message, so if retrying is
+		// successful, the next logical chunk will be returned.
+		msg, err = r.reopenStream()
+	}
+
+	return msg, err
+}
+
+// reopenStream "closes" the existing stream and attempts to reopen a stream and
+// sets the Reader's stream and cancelStream properties in the process.
+func (r *gRPCReader) reopenStream() (*storagepb.ReadObjectResponse, error) {
+	// Close existing stream and initialize new stream with updated offset.
+	r.Close()
+
+	res, cancel, err := r.reopen(r.seen)
+	if err != nil {
+		return nil, err
+	}
+	r.stream = res.stream
+	r.cancel = cancel
+	return res.response, nil
+}
+
+func newGRPCWriter(c *grpcStorageClient, params *openWriterParams, r io.Reader) *gRPCWriter {
+	size := params.chunkSize
+	if params.chunkSize == 0 {
+		// TODO: Should we actually use the minimum of 256 KB here when the user
+		// indicates they want minimal memory usage? We cannot do a zero-copy,
+		// bufferless upload like HTTP/JSON can.
+		// TODO: We need to determine if we can avoid starting a
+		// resumable upload when the user *plans* to send more than bufSize but
+		// with a bufferless upload.
+		size = maxPerMessageWriteSize
+	}
+
+	return &gRPCWriter{
+		buf:           make([]byte, size),
+		c:             c,
+		ctx:           params.ctx,
+		reader:        r,
+		bucket:        params.bucket,
+		attrs:         params.attrs,
+		conds:         params.conds,
+		encryptionKey: params.encryptionKey,
+		sendCRC32C:    params.sendCRC32C,
+	}
+}
+
+// gRPCWriter is a wrapper around the the gRPC client-stream API that manages
+// sending chunks of data provided by the user over the stream.
+type gRPCWriter struct {
+	c      *grpcStorageClient
+	buf    []byte
+	reader io.Reader
+
+	ctx context.Context
+
+	bucket        string
+	attrs         *ObjectAttrs
+	conds         *Conditions
+	encryptionKey []byte
+
+	sendCRC32C bool
+
+	// The gRPC client-stream used for sending buffers.
+	stream storagepb.Storage_WriteObjectClient
+
+	// The Resumable Upload ID started by a gRPC-based Writer.
+	upid string
+}
+
+// startResumableUpload initializes a Resumable Upload with gRPC and sets the
+// upload ID on the Writer.
+func (w *gRPCWriter) startResumableUpload() error {
+	spec, err := w.writeObjectSpec()
+	if err != nil {
+		return err
+	}
+	upres, err := w.c.raw.StartResumableWrite(w.ctx, &storagepb.StartResumableWriteRequest{
+		WriteObjectSpec: spec,
+	})
+
+	w.upid = upres.GetUploadId()
+	return err
+}
+
+// queryProgress is a helper that queries the status of the resumable upload
+// associated with the given upload ID.
+func (w *gRPCWriter) queryProgress() (int64, error) {
+	q, err := w.c.raw.QueryWriteStatus(w.ctx, &storagepb.QueryWriteStatusRequest{UploadId: w.upid})
+
+	// q.GetCommittedSize() will return 0 if q is nil.
+	return q.GetPersistedSize(), err
+}
+
+// uploadBuffer opens a Write stream and uploads the buffer at the given offset (if
+// uploading a chunk for a resumable uploadBuffer), and will mark the write as
+// finished if we are done receiving data from the user. The resulting write
+// offset after uploading the buffer is returned, as well as a boolean
+// indicating if the Object has been finalized. If it has been finalized, the
+// final Object will be returned as well. Finalizing the upload is primarily
+// important for Resumable Uploads. A simple or multi-part upload will always
+// be finalized once the entire buffer has been written.
+func (w *gRPCWriter) uploadBuffer(recvd int, start int64, doneReading bool) (*storagepb.Object, int64, bool, error) {
+	var err error
+	var finishWrite bool
+	var sent, limit int = 0, maxPerMessageWriteSize
+	offset := start
+	toWrite := w.buf[:recvd]
+	for {
+		first := sent == 0
+		// This indicates that this is the last message and the remaining
+		// data fits in one message.
+		belowLimit := recvd-sent <= limit
+		if belowLimit {
+			limit = recvd - sent
+		}
+		if belowLimit && doneReading {
+			finishWrite = true
+		}
+
+		// Prepare chunk section for upload.
+		data := toWrite[sent : sent+limit]
+		req := &storagepb.WriteObjectRequest{
+			Data: &storagepb.WriteObjectRequest_ChecksummedData{
+				ChecksummedData: &storagepb.ChecksummedData{
+					Content: data,
+				},
+			},
+			WriteOffset: offset,
+			FinishWrite: finishWrite,
+		}
+
+		// Open a new stream and set the first_message field on the request.
+		// The first message on the WriteObject stream must either be the
+		// Object or the Resumable Upload ID.
+		if first {
+			w.stream, err = w.c.raw.WriteObject(w.ctx)
+			if err != nil {
+				return nil, 0, false, err
+			}
+
+			if w.upid != "" {
+				req.FirstMessage = &storagepb.WriteObjectRequest_UploadId{UploadId: w.upid}
+			} else {
+				spec, err := w.writeObjectSpec()
+				if err != nil {
+					return nil, 0, false, err
+				}
+				req.FirstMessage = &storagepb.WriteObjectRequest_WriteObjectSpec{
+					WriteObjectSpec: spec,
+				}
+			}
+
+			// TODO: Currently the checksums are only sent on the first message
+			// of the stream, but in the future, we must also support sending it
+			// on the *last* message of the stream (instead of the first).
+			if w.sendCRC32C {
+				req.ObjectChecksums = &storagepb.ObjectChecksums{
+					Crc32C:  proto.Uint32(w.attrs.CRC32C),
+					Md5Hash: w.attrs.MD5,
+				}
+			}
+		}
+
+		err = w.stream.Send(req)
+		if err == io.EOF {
+			// err was io.EOF. The client-side of a stream only gets an EOF on Send
+			// when the backend closes the stream and wants to return an error
+			// status. Closing the stream receives the status as an error.
+			_, err = w.stream.CloseAndRecv()
+
+			// Retriable errors mean we should start over and attempt to
+			// resend the entire buffer via a new stream.
+			// If not retriable, falling through will return the error received
+			// from closing the stream.
+			if shouldRetry(err) {
+				sent = 0
+				finishWrite = false
+				// TODO: Add test case for failure modes of querying progress.
+				offset, err = w.determineOffset(start)
+				if err == nil {
+					continue
+				}
+			}
+		}
+		if err != nil {
+			return nil, 0, false, err
+		}
+
+		// Update the immediate stream's sent total and the upload offset with
+		// the data sent.
+		sent += len(data)
+		offset += int64(len(data))
+
+		// Not done sending data, do not attempt to commit it yet, loop around
+		// and send more data.
+		if recvd-sent > 0 {
+			continue
+		}
+
+		// Done sending data. Close the stream to "commit" the data sent.
+		resp, finalized, err := w.commit()
+		// Retriable errors mean we should start over and attempt to
+		// resend the entire buffer via a new stream.
+		// If not retriable, falling through will return the error received
+		// from closing the stream.
+		if shouldRetry(err) {
+			sent = 0
+			finishWrite = false
+			offset, err = w.determineOffset(start)
+			if err == nil {
+				continue
+			}
+		}
+		if err != nil {
+			return nil, 0, false, err
+		}
+
+		return resp.GetResource(), offset, finalized, nil
+	}
+}
+
+// determineOffset either returns the offset given to it in the case of a simple
+// upload, or queries the write status in the case a resumable upload is being
+// used.
+func (w *gRPCWriter) determineOffset(offset int64) (int64, error) {
+	// For a Resumable Upload, we must start from however much data
+	// was committed.
+	if w.upid != "" {
+		committed, err := w.queryProgress()
+		if err != nil {
+			return 0, err
+		}
+		offset = committed
+	}
+	return offset, nil
+}
+
+// commit closes the stream to commit the data sent and potentially receive
+// the finalized object if finished uploading. If the last request sent
+// indicated that writing was finished, the Object will be finalized and
+// returned. If not, then the Object will be nil, and the boolean returned will
+// be false.
+func (w *gRPCWriter) commit() (*storagepb.WriteObjectResponse, bool, error) {
+	finalized := true
+	resp, err := w.stream.CloseAndRecv()
+	if err == io.EOF {
+		// Closing a stream for a resumable upload finish_write = false results
+		// in an EOF which can be ignored, as we aren't done uploading yet.
+		finalized = false
+		err = nil
+	}
+	// Drop the stream reference as it has been closed.
+	w.stream = nil
+
+	return resp, finalized, err
+}
+
+// writeObjectSpec constructs a WriteObjectSpec proto using the Writer's
+// ObjectAttrs and applies its Conditions. This is only used for gRPC.
+func (w *gRPCWriter) writeObjectSpec() (*storagepb.WriteObjectSpec, error) {
+	// To avoid modifying the ObjectAttrs embeded in the calling writer, deref
+	// the ObjectAttrs pointer to make a copy, then assign the desired name to
+	// the attribute.
+	attrs := *w.attrs
+
+	spec := &storagepb.WriteObjectSpec{
+		Resource: attrs.toProtoObject(w.bucket),
+	}
+	// WriteObject doesn't support the generation condition, so use -1.
+	if err := applyCondsProto("WriteObject", -1, w.conds, spec); err != nil {
+		return nil, err
+	}
+	return spec, nil
+}
+
+// read copies the data in the reader to the given buffer and reports how much
+// data was read into the buffer and if there is no more data to read (EOF).
+func (w *gRPCWriter) read() (int, bool, error) {
+	// Set n to -1 to start the Read loop.
+	var n, recvd int = -1, 0
+	var err error
+	for err == nil && n != 0 {
+		// The routine blocks here until data is received.
+		n, err = w.reader.Read(w.buf[recvd:])
+		recvd += n
+	}
+	var done bool
+	if err == io.EOF {
+		done = true
+		err = nil
+	}
+	return recvd, done, err
+}
+
+func checkCanceled(err error) error {
+	if status.Code(err) == codes.Canceled {
+		return context.Canceled
+	}
+
+	return err
+}
diff --git a/vendor/cloud.google.com/go/storage/http_client.go b/vendor/cloud.google.com/go/storage/http_client.go
index 1ec55150f5..45c175f898 100644
--- a/vendor/cloud.google.com/go/storage/http_client.go
+++ b/vendor/cloud.google.com/go/storage/http_client.go
@@ -16,14 +16,21 @@ package storage
 
 import (
 	"context"
+	"encoding/base64"
 	"errors"
 	"fmt"
+	"io"
+	"io/ioutil"
 	"net/http"
 	"net/url"
 	"os"
 	"reflect"
+	"strconv"
 	"strings"
+	"time"
 
+	"cloud.google.com/go/internal/optional"
+	"cloud.google.com/go/internal/trace"
 	"golang.org/x/oauth2/google"
 	"google.golang.org/api/googleapi"
 	"google.golang.org/api/iterator"
@@ -378,14 +385,143 @@ func (c *httpStorageClient) ListObjects(ctx context.Context, bucket string, q *Q
 
 // Object metadata methods.
 
-func (c *httpStorageClient) DeleteObject(ctx context.Context, bucket, object string, conds *Conditions, opts ...storageOption) error {
-	return errMethodNotSupported
+func (c *httpStorageClient) DeleteObject(ctx context.Context, bucket, object string, gen int64, conds *Conditions, opts ...storageOption) error {
+	s := callSettings(c.settings, opts...)
+	req := c.raw.Objects.Delete(bucket, object).Context(ctx)
+	if err := applyConds("Delete", gen, conds, req); err != nil {
+		return err
+	}
+	if s.userProject != "" {
+		req.UserProject(s.userProject)
+	}
+	err := run(ctx, func() error { return req.Context(ctx).Do() }, s.retry, s.idempotent, setRetryHeaderHTTP(req))
+	var e *googleapi.Error
+	if ok := errors.As(err, &e); ok && e.Code == http.StatusNotFound {
+		return ErrObjectNotExist
+	}
+	return err
 }
-func (c *httpStorageClient) GetObject(ctx context.Context, bucket, object string, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
-	return nil, errMethodNotSupported
+
+func (c *httpStorageClient) GetObject(ctx context.Context, bucket, object string, gen int64, encryptionKey []byte, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
+	s := callSettings(c.settings, opts...)
+	req := c.raw.Objects.Get(bucket, object).Projection("full").Context(ctx)
+	if err := applyConds("Attrs", gen, conds, req); err != nil {
+		return nil, err
+	}
+	if s.userProject != "" {
+		req.UserProject(s.userProject)
+	}
+	if err := setEncryptionHeaders(req.Header(), encryptionKey, false); err != nil {
+		return nil, err
+	}
+	var obj *raw.Object
+	var err error
+	err = run(ctx, func() error {
+		obj, err = req.Context(ctx).Do()
+		return err
+	}, s.retry, s.idempotent, setRetryHeaderHTTP(req))
+	var e *googleapi.Error
+	if ok := errors.As(err, &e); ok && e.Code == http.StatusNotFound {
+		return nil, ErrObjectNotExist
+	}
+	if err != nil {
+		return nil, err
+	}
+	return newObject(obj), nil
 }
-func (c *httpStorageClient) UpdateObject(ctx context.Context, bucket, object string, uattrs *ObjectAttrsToUpdate, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
-	return nil, errMethodNotSupported
+
+func (c *httpStorageClient) UpdateObject(ctx context.Context, bucket, object string, uattrs *ObjectAttrsToUpdate, gen int64, encryptionKey []byte, conds *Conditions, opts ...storageOption) (*ObjectAttrs, error) {
+	s := callSettings(c.settings, opts...)
+
+	var attrs ObjectAttrs
+	// Lists of fields to send, and set to null, in the JSON.
+	var forceSendFields, nullFields []string
+	if uattrs.ContentType != nil {
+		attrs.ContentType = optional.ToString(uattrs.ContentType)
+		// For ContentType, sending the empty string is a no-op.
+		// Instead we send a null.
+		if attrs.ContentType == "" {
+			nullFields = append(nullFields, "ContentType")
+		} else {
+			forceSendFields = append(forceSendFields, "ContentType")
+		}
+	}
+	if uattrs.ContentLanguage != nil {
+		attrs.ContentLanguage = optional.ToString(uattrs.ContentLanguage)
+		// For ContentLanguage it's an error to send the empty string.
+		// Instead we send a null.
+		if attrs.ContentLanguage == "" {
+			nullFields = append(nullFields, "ContentLanguage")
+		} else {
+			forceSendFields = append(forceSendFields, "ContentLanguage")
+		}
+	}
+	if uattrs.ContentEncoding != nil {
+		attrs.ContentEncoding = optional.ToString(uattrs.ContentEncoding)
+		forceSendFields = append(forceSendFields, "ContentEncoding")
+	}
+	if uattrs.ContentDisposition != nil {
+		attrs.ContentDisposition = optional.ToString(uattrs.ContentDisposition)
+		forceSendFields = append(forceSendFields, "ContentDisposition")
+	}
+	if uattrs.CacheControl != nil {
+		attrs.CacheControl = optional.ToString(uattrs.CacheControl)
+		forceSendFields = append(forceSendFields, "CacheControl")
+	}
+	if uattrs.EventBasedHold != nil {
+		attrs.EventBasedHold = optional.ToBool(uattrs.EventBasedHold)
+		forceSendFields = append(forceSendFields, "EventBasedHold")
+	}
+	if uattrs.TemporaryHold != nil {
+		attrs.TemporaryHold = optional.ToBool(uattrs.TemporaryHold)
+		forceSendFields = append(forceSendFields, "TemporaryHold")
+	}
+	if !uattrs.CustomTime.IsZero() {
+		attrs.CustomTime = uattrs.CustomTime
+		forceSendFields = append(forceSendFields, "CustomTime")
+	}
+	if uattrs.Metadata != nil {
+		attrs.Metadata = uattrs.Metadata
+		if len(attrs.Metadata) == 0 {
+			// Sending the empty map is a no-op. We send null instead.
+			nullFields = append(nullFields, "Metadata")
+		} else {
+			forceSendFields = append(forceSendFields, "Metadata")
+		}
+	}
+	if uattrs.ACL != nil {
+		attrs.ACL = uattrs.ACL
+		// It's an error to attempt to delete the ACL, so
+		// we don't append to nullFields here.
+		forceSendFields = append(forceSendFields, "Acl")
+	}
+	rawObj := attrs.toRawObject(bucket)
+	rawObj.ForceSendFields = forceSendFields
+	rawObj.NullFields = nullFields
+	call := c.raw.Objects.Patch(bucket, object, rawObj).Projection("full").Context(ctx)
+	if err := applyConds("Update", gen, conds, call); err != nil {
+		return nil, err
+	}
+	if s.userProject != "" {
+		call.UserProject(s.userProject)
+	}
+	if uattrs.PredefinedACL != "" {
+		call.PredefinedAcl(uattrs.PredefinedACL)
+	}
+	if err := setEncryptionHeaders(call.Header(), encryptionKey, false); err != nil {
+		return nil, err
+	}
+	var obj *raw.Object
+	var err error
+	err = run(ctx, func() error { obj, err = call.Do(); return err }, s.retry, s.idempotent, setRetryHeaderHTTP(call))
+	var e *googleapi.Error
+	if errors.As(err, &e) && e.Code == http.StatusNotFound {
+		return nil, ErrObjectNotExist
+	}
+	if err != nil {
+		return nil, err
+	}
+	return newObject(obj), nil
 }
 
 // Default Object ACL methods.
@@ -479,9 +615,25 @@ func configureACLCall(ctx context.Context, userProject string, call interface{ H
 func (c *httpStorageClient) DeleteObjectACL(ctx context.Context, bucket, object string, entity ACLEntity, opts ...storageOption) error {
 	return errMethodNotSupported
 }
+
+// ListObjectACLs retrieves object ACL entries. By default, it operates on the latest generation of this object.
+// Selecting a specific generation of this object is not currently supported by the client.
 func (c *httpStorageClient) ListObjectACLs(ctx context.Context, bucket, object string, opts ...storageOption) ([]ACLRule, error) {
-	return nil, errMethodNotSupported
+	s := callSettings(c.settings, opts...)
+	var acls *raw.ObjectAccessControls
+	var err error
+	req := c.raw.ObjectAccessControls.List(bucket, object)
+	configureACLCall(ctx, s.userProject, req)
+	err = run(ctx, func() error {
+		acls, err = req.Do()
+		return err
+	}, s.retry, s.idempotent, setRetryHeaderHTTP(req))
+	if err != nil {
+		return nil, err
+	}
+	return toObjectACLRules(acls.Items), nil
 }
+
 func (c *httpStorageClient) UpdateObjectACL(ctx context.Context, bucket, object string, entity ACLEntity, role ACLRole, opts ...storageOption) (*ACLRule, error) {
 	return nil, errMethodNotSupported
 }
@@ -495,11 +647,305 @@ func (c *httpStorageClient) RewriteObject(ctx context.Context, req *rewriteObjec
 	return nil, errMethodNotSupported
 }
 
-func (c *httpStorageClient) OpenReader(ctx context.Context, r *Reader, opts ...storageOption) error {
-	return errMethodNotSupported
+func (c *httpStorageClient) NewRangeReader(ctx context.Context, params *newRangeReaderParams, opts ...storageOption) (r *Reader, err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.httpStorageClient.NewRangeReader")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	s := callSettings(c.settings, opts...)
+
+	if params.offset < 0 && params.length >= 0 {
+		return nil, fmt.Errorf("storage: invalid offset %d < 0 requires negative length", params.offset)
+	}
+	if params.conds != nil {
+		if err := params.conds.validate("NewRangeReader"); err != nil {
+			return nil, err
+		}
+	}
+	u := &url.URL{
+		Scheme: c.scheme,
+		Host:   c.readHost,
+		Path:   fmt.Sprintf("/%s/%s", params.bucket, params.object),
+	}
+	verb := "GET"
+	if params.length == 0 {
+		verb = "HEAD"
+	}
+	req, err := http.NewRequest(verb, u.String(), nil)
+	if err != nil {
+		return nil, err
+	}
+	req = req.WithContext(ctx)
+	if s.userProject != "" {
+		req.Header.Set("X-Goog-User-Project", s.userProject)
+	}
+	// TODO(noahdietz): add option for readCompressed.
+	// if o.readCompressed {
+	// 	req.Header.Set("Accept-Encoding", "gzip")
+	// }
+	if err := setEncryptionHeaders(req.Header, params.encryptionKey, false); err != nil {
+		return nil, err
+	}
+
+	// Define a function that initiates a Read with offset and length, assuming we
+	// have already read seen bytes.
+	reopen := func(seen int64) (*http.Response, error) {
+		// If the context has already expired, return immediately without making a
+		// call.
+		if err := ctx.Err(); err != nil {
+			return nil, err
+		}
+		start := params.offset + seen
+		if params.length < 0 && start < 0 {
+			req.Header.Set("Range", fmt.Sprintf("bytes=%d", start))
+		} else if params.length < 0 && start > 0 {
+			req.Header.Set("Range", fmt.Sprintf("bytes=%d-", start))
+		} else if params.length > 0 {
+			// The end character isn't affected by how many bytes we've seen.
+			req.Header.Set("Range", fmt.Sprintf("bytes=%d-%d", start, params.offset+params.length-1))
+		}
+		// We wait to assign conditions here because the generation number can change in between reopen() runs.
+		if err := setConditionsHeaders(req.Header, params.conds); err != nil {
+			return nil, err
+		}
+		// If an object generation is specified, include generation as query string parameters.
+		if params.gen >= 0 {
+			req.URL.RawQuery = fmt.Sprintf("generation=%d", params.gen)
+		}
+
+		var res *http.Response
+		err = run(ctx, func() error {
+			res, err = c.hc.Do(req)
+			if err != nil {
+				return err
+			}
+			if res.StatusCode == http.StatusNotFound {
+				res.Body.Close()
+				return ErrObjectNotExist
+			}
+			if res.StatusCode < 200 || res.StatusCode > 299 {
+				body, _ := ioutil.ReadAll(res.Body)
+				res.Body.Close()
+				return &googleapi.Error{
+					Code:   res.StatusCode,
+					Header: res.Header,
+					Body:   string(body),
+				}
+			}
+
+			partialContentNotSatisfied :=
+				!decompressiveTranscoding(res) &&
+					start > 0 && params.length != 0 &&
+					res.StatusCode != http.StatusPartialContent
+
+			if partialContentNotSatisfied {
+				res.Body.Close()
+				return errors.New("storage: partial request not satisfied")
+			}
+
+			// With "Content-Encoding": "gzip" aka decompressive transcoding, GCS serves
+			// back the whole file regardless of the range count passed in as per:
+			//      https://cloud.google.com/storage/docs/transcoding#range,
+			// thus we have to manually move the body forward by seen bytes.
+			if decompressiveTranscoding(res) && seen > 0 {
+				_, _ = io.CopyN(ioutil.Discard, res.Body, seen)
+			}
+
+			// If a generation hasn't been specified, and this is the first response we get, let's record the
+			// generation. In future requests we'll use this generation as a precondition to avoid data races.
+			if params.gen < 0 && res.Header.Get("X-Goog-Generation") != "" {
+				gen64, err := strconv.ParseInt(res.Header.Get("X-Goog-Generation"), 10, 64)
+				if err != nil {
+					return err
+				}
+				params.gen = gen64
+			}
+			return nil
+		}, s.retry, s.idempotent, setRetryHeaderHTTP(nil))
+		if err != nil {
+			return nil, err
+		}
+		return res, nil
+	}
+
+	res, err := reopen(0)
+	if err != nil {
+		return nil, err
+	}
+	var (
+		size        int64 // total size of object, even if a range was requested.
+		checkCRC    bool
+		crc         uint32
+		startOffset int64 // non-zero if range request.
+	)
+	if res.StatusCode == http.StatusPartialContent {
+		cr := strings.TrimSpace(res.Header.Get("Content-Range"))
+		if !strings.HasPrefix(cr, "bytes ") || !strings.Contains(cr, "/") {
+			return nil, fmt.Errorf("storage: invalid Content-Range %q", cr)
+		}
+		// Content range is formatted <first byte>-<last byte>/<total size>. We take
+		// the total size.
+		size, err = strconv.ParseInt(cr[strings.LastIndex(cr, "/")+1:], 10, 64)
+		if err != nil {
+			return nil, fmt.Errorf("storage: invalid Content-Range %q", cr)
+		}
+
+		dashIndex := strings.Index(cr, "-")
+		if dashIndex >= 0 {
+			startOffset, err = strconv.ParseInt(cr[len("bytes="):dashIndex], 10, 64)
+			if err != nil {
+				return nil, fmt.Errorf("storage: invalid Content-Range %q: %v", cr, err)
+			}
+		}
+	} else {
+		size = res.ContentLength
+		// Check the CRC iff all of the following hold:
+		// - We asked for content (length != 0).
+		// - We got all the content (status != PartialContent).
+		// - The server sent a CRC header.
+		// - The Go http stack did not uncompress the file.
+		// - We were not served compressed data that was uncompressed on download.
+		// The problem with the last two cases is that the CRC will not match -- GCS
+		// computes it on the compressed contents, but we compute it on the
+		// uncompressed contents.
+		if params.length != 0 && !res.Uncompressed && !uncompressedByServer(res) {
+			crc, checkCRC = parseCRC32c(res)
+		}
+	}
+
+	remain := res.ContentLength
+	body := res.Body
+	if params.length == 0 {
+		remain = 0
+		body.Close()
+		body = emptyBody
+	}
+	var metaGen int64
+	if res.Header.Get("X-Goog-Metageneration") != "" {
+		metaGen, err = strconv.ParseInt(res.Header.Get("X-Goog-Metageneration"), 10, 64)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var lm time.Time
+	if res.Header.Get("Last-Modified") != "" {
+		lm, err = http.ParseTime(res.Header.Get("Last-Modified"))
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	attrs := ReaderObjectAttrs{
+		Size:            size,
+		ContentType:     res.Header.Get("Content-Type"),
+		ContentEncoding: res.Header.Get("Content-Encoding"),
+		CacheControl:    res.Header.Get("Cache-Control"),
+		LastModified:    lm,
+		StartOffset:     startOffset,
+		Generation:      params.gen,
+		Metageneration:  metaGen,
+	}
+	return &Reader{
+		Attrs:    attrs,
+		size:     size,
+		remain:   remain,
+		wantCRC:  crc,
+		checkCRC: checkCRC,
+		reader: &httpReader{
+			reopen: reopen,
+			body:   body,
+		},
+	}, nil
 }
-func (c *httpStorageClient) OpenWriter(ctx context.Context, w *Writer, opts ...storageOption) error {
-	return errMethodNotSupported
+
+func (c *httpStorageClient) OpenWriter(params *openWriterParams, opts ...storageOption) (*io.PipeWriter, error) {
+	s := callSettings(c.settings, opts...)
+	errorf := params.setError
+	setObj := params.setObj
+	progress := params.progress
+	attrs := params.attrs
+
+	mediaOpts := []googleapi.MediaOption{
+		googleapi.ChunkSize(params.chunkSize),
+	}
+	if c := attrs.ContentType; c != "" {
+		mediaOpts = append(mediaOpts, googleapi.ContentType(c))
+	}
+	if params.chunkRetryDeadline != 0 {
+		mediaOpts = append(mediaOpts, googleapi.ChunkRetryDeadline(params.chunkRetryDeadline))
+	}
+
+	pr, pw := io.Pipe()
+
+	go func() {
+		defer close(params.donec)
+
+		rawObj := attrs.toRawObject(params.bucket)
+		if params.sendCRC32C {
+			rawObj.Crc32c = encodeUint32(attrs.CRC32C)
+		}
+		if attrs.MD5 != nil {
+			rawObj.Md5Hash = base64.StdEncoding.EncodeToString(attrs.MD5)
+		}
+		call := c.raw.Objects.Insert(params.bucket, rawObj).
+			Media(pr, mediaOpts...).
+			Projection("full").
+			Context(params.ctx).
+			Name(params.attrs.Name)
+		call.ProgressUpdater(func(n, _ int64) { progress(n) })
+
+		if attrs.KMSKeyName != "" {
+			call.KmsKeyName(attrs.KMSKeyName)
+		}
+		if attrs.PredefinedACL != "" {
+			call.PredefinedAcl(attrs.PredefinedACL)
+		}
+		if err := setEncryptionHeaders(call.Header(), params.encryptionKey, false); err != nil {
+			errorf(err)
+			pr.CloseWithError(err)
+			return
+		}
+		var resp *raw.Object
+		err := applyConds("NewWriter", params.attrs.Generation, params.conds, call)
+		if err == nil {
+			if s.userProject != "" {
+				call.UserProject(s.userProject)
+			}
+			// TODO(tritone): Remove this code when Uploads begin to support
+			// retry attempt header injection with "client header" injection.
+			setClientHeader(call.Header())
+
+			// The internals that perform call.Do automatically retry both the initial
+			// call to set up the upload as well as calls to upload individual chunks
+			// for a resumable upload (as long as the chunk size is non-zero). Hence
+			// there is no need to add retries here.
+
+			// Retry only when the operation is idempotent or the retry policy is RetryAlways.
+			isIdempotent := params.conds != nil && (params.conds.GenerationMatch >= 0 || params.conds.DoesNotExist == true)
+			var useRetry bool
+			if (s.retry == nil || s.retry.policy == RetryIdempotent) && isIdempotent {
+				useRetry = true
+			} else if s.retry != nil && s.retry.policy == RetryAlways {
+				useRetry = true
+			}
+			if useRetry {
+				if s.retry != nil {
+					call.WithRetry(s.retry.backoff, s.retry.shouldRetry)
+				} else {
+					call.WithRetry(nil, nil)
+				}
+			}
+			resp, err = call.Do()
+		}
+		if err != nil {
+			errorf(err)
+			pr.CloseWithError(err)
+			return
+		}
+		setObj(newObject(resp))
+	}()
+
+	return pw, nil
 }
 
 // IAM methods.
@@ -575,3 +1021,96 @@ func (c *httpStorageClient) CreateHMACKey(ctx context.Context, desc *hmacKeyDesc
 func (c *httpStorageClient) DeleteHMACKey(ctx context.Context, desc *hmacKeyDesc, opts ...storageOption) error {
 	return errMethodNotSupported
 }
+
+// Notification methods.
+
+// ListNotifications returns all the Notifications configured for this bucket, as a map indexed by notification ID.
+//
+// Note: This API does not support pagination. However, entity limits cap the number of notifications on a single bucket,
+// so all results will be returned in the first response. See https://cloud.google.com/storage/quotas#buckets.
+func (c *httpStorageClient) ListNotifications(ctx context.Context, bucket string, opts ...storageOption) (n map[string]*Notification, err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.httpStorageClient.ListNotifications")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	s := callSettings(c.settings, opts...)
+	call := c.raw.Notifications.List(bucket)
+	if s.userProject != "" {
+		call.UserProject(s.userProject)
+	}
+	var res *raw.Notifications
+	err = run(ctx, func() error {
+		res, err = call.Context(ctx).Do()
+		return err
+	}, s.retry, true, setRetryHeaderHTTP(call))
+	if err != nil {
+		return nil, err
+	}
+	return notificationsToMap(res.Items), nil
+}
+
+func (c *httpStorageClient) CreateNotification(ctx context.Context, bucket string, n *Notification, opts ...storageOption) (ret *Notification, err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.httpStorageClient.CreateNotification")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	s := callSettings(c.settings, opts...)
+	call := c.raw.Notifications.Insert(bucket, toRawNotification(n))
+	if s.userProject != "" {
+		call.UserProject(s.userProject)
+	}
+	var rn *raw.Notification
+	err = run(ctx, func() error {
+		rn, err = call.Context(ctx).Do()
+		return err
+	}, s.retry, s.idempotent, setRetryHeaderHTTP(call))
+	if err != nil {
+		return nil, err
+	}
+	return toNotification(rn), nil
+}
+
+func (c *httpStorageClient) DeleteNotification(ctx context.Context, bucket string, id string, opts ...storageOption) (err error) {
+	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.httpStorageClient.DeleteNotification")
+	defer func() { trace.EndSpan(ctx, err) }()
+
+	s := callSettings(c.settings, opts...)
+	call := c.raw.Notifications.Delete(bucket, id)
+	if s.userProject != "" {
+		call.UserProject(s.userProject)
+	}
+	return run(ctx, func() error {
+		return call.Context(ctx).Do()
+	}, s.retry, s.idempotent, setRetryHeaderHTTP(call))
+}
+
+type httpReader struct {
+	body   io.ReadCloser
+	seen   int64
+	reopen func(seen int64) (*http.Response, error)
+}
+
+func (r *httpReader) Read(p []byte) (int, error) {
+	n := 0
+	for len(p[n:]) > 0 {
+		m, err := r.body.Read(p[n:])
+		n += m
+		r.seen += int64(m)
+		if err == nil || err == io.EOF {
+			return n, err
+		}
+		// Read failed (likely due to connection issues), but we will try to reopen
+		// the pipe and continue. Send a ranged read request that takes into account
+		// the number of bytes we've already seen.
+		res, err := r.reopen(r.seen)
+		if err != nil {
+			// reopen already retries
+			return n, err
+		}
+		r.body.Close()
+		r.body = res.Body
+	}
+	return n, nil
+}
+
+func (r *httpReader) Close() error {
+	return r.body.Close()
+}
diff --git a/vendor/cloud.google.com/go/storage/internal/apiv2/storage_client.go b/vendor/cloud.google.com/go/storage/internal/apiv2/storage_client.go
index 803f9d2602..ced732ca69 100644
--- a/vendor/cloud.google.com/go/storage/internal/apiv2/storage_client.go
+++ b/vendor/cloud.google.com/go/storage/internal/apiv2/storage_client.go
@@ -113,7 +113,7 @@ func defaultCallOptions() *CallOptions {
 	}
 }
 
-// internalClient is an interface that defines the methods availaible from Cloud Storage API.
+// internalClient is an interface that defines the methods available from Cloud Storage API.
 type internalClient interface {
 	Close() error
 	setGoogleClientInfo(...string)
@@ -312,13 +312,40 @@ func (c *Client) UpdateObject(ctx context.Context, req *storagepb.UpdateObjectRe
 // true, or else it is an error.
 //
 // For a resumable write, the client should instead call
-// StartResumableWrite() and provide that method an WriteObjectSpec.
+// StartResumableWrite(), populating a WriteObjectSpec into that request.
 // They should then attach the returned upload_id to the first message of
-// each following call to Create. If there is an error or the connection is
-// broken during the resumable Create(), the client should check the status
-// of the Create() by calling QueryWriteStatus() and continue writing from
-// the returned persisted_size. This may be less than the amount of data the
-// client previously sent.
+// each following call to WriteObject. If the stream is closed before
+// finishing the upload (either explicitly by the client or due to a network
+// error or an error response from the server), the client should do as
+// follows:
+//
+//   Check the result Status of the stream, to determine if writing can be
+//   resumed on this stream or must be restarted from scratch (by calling
+//   StartResumableWrite()). The resumable errors are DEADLINE_EXCEEDED,
+//   INTERNAL, and UNAVAILABLE. For each case, the client should use binary
+//   exponential backoff before retrying.  Additionally, writes can be
+//   resumed after RESOURCE_EXHAUSTED errors, but only after taking
+//   appropriate measures, which may include reducing aggregate send rate
+//   across clients and/or requesting a quota increase for your project.
+//
+//   If the call to WriteObject returns ABORTED, that indicates
+//   concurrent attempts to update the resumable write, caused either by
+//   multiple racing clients or by a single client where the previous
+//   request was timed out on the client side but nonetheless reached the
+//   server. In this case the client should take steps to prevent further
+//   concurrent writes (e.g., increase the timeouts, stop using more than
+//   one process to perform the upload, etc.), and then should follow the
+//   steps below for resuming the upload.
+//
+//   For resumable errors, the client should call QueryWriteStatus() and
+//   then continue writing from the returned persisted_size. This may be
+//   less than the amount of data the client previously sent. Note also that
+//   it is acceptable to send data starting at an offset earlier than the
+//   returned persisted_size; in this case, the service will skip data at
+//   offsets that were already persisted (without checking that it matches
+//   the previously written data), and write only the data starting from the
+//   persisted offset. This behavior can make client-side handling simpler
+//   in some cases.
 //
 // The service will not view the object as complete until the client has
 // sent a WriteObjectRequest with finish_write set to true. Sending any
@@ -326,6 +353,10 @@ func (c *Client) UpdateObject(ctx context.Context, req *storagepb.UpdateObjectRe
 // true will cause an error. The client should check the response it
 // receives to determine how much data the service was able to commit and
 // whether the service views the object as complete.
+//
+// Attempting to resume an already finalized object will result in an OK
+// status, with a WriteObjectResponse containing the finalized object’s
+// metadata.
 func (c *Client) WriteObject(ctx context.Context, opts ...gax.CallOption) (storagepb.Storage_WriteObjectClient, error) {
 	return c.internalClient.WriteObject(ctx, opts...)
 }
diff --git a/vendor/cloud.google.com/go/storage/internal/version.go b/vendor/cloud.google.com/go/storage/internal/version.go
index 08bddba748..6395537003 100644
--- a/vendor/cloud.google.com/go/storage/internal/version.go
+++ b/vendor/cloud.google.com/go/storage/internal/version.go
@@ -15,4 +15,4 @@
 package internal
 
 // Version is the current tagged release of the library.
-const Version = "1.22.1"
+const Version = "1.23.0"
diff --git a/vendor/cloud.google.com/go/storage/notifications.go b/vendor/cloud.google.com/go/storage/notifications.go
index dd43822b6a..fe3e3ae069 100644
--- a/vendor/cloud.google.com/go/storage/notifications.go
+++ b/vendor/cloud.google.com/go/storage/notifications.go
@@ -22,6 +22,7 @@ import (
 
 	"cloud.google.com/go/internal/trace"
 	raw "google.golang.org/api/storage/v1"
+	storagepb "google.golang.org/genproto/googleapis/storage/v2"
 )
 
 // A Notification describes how to send Cloud PubSub messages when certain
@@ -91,6 +92,30 @@ func toNotification(rn *raw.Notification) *Notification {
 	return n
 }
 
+func toNotificationFromProto(pbn *storagepb.Notification) *Notification {
+	n := &Notification{
+		ID:               pbn.GetName(),
+		EventTypes:       pbn.GetEventTypes(),
+		ObjectNamePrefix: pbn.GetObjectNamePrefix(),
+		CustomAttributes: pbn.GetCustomAttributes(),
+		PayloadFormat:    pbn.GetPayloadFormat(),
+	}
+	n.TopicProjectID, n.TopicID = parseNotificationTopic(pbn.Topic)
+	return n
+}
+
+func toProtoNotification(n *Notification) *storagepb.Notification {
+	return &storagepb.Notification{
+		Name: n.ID,
+		Topic: fmt.Sprintf("//pubsub.googleapis.com/projects/%s/topics/%s",
+			n.TopicProjectID, n.TopicID),
+		EventTypes:       n.EventTypes,
+		ObjectNamePrefix: n.ObjectNamePrefix,
+		CustomAttributes: n.CustomAttributes,
+		PayloadFormat:    n.PayloadFormat,
+	}
+}
+
 var topicRE = regexp.MustCompile("^//pubsub.googleapis.com/projects/([^/]+)/topics/([^/]+)")
 
 // parseNotificationTopic extracts the project and topic IDs from from the full
@@ -179,6 +204,14 @@ func notificationsToMap(rns []*raw.Notification) map[string]*Notification {
 	return m
 }
 
+func notificationsToMapFromProto(ns []*storagepb.Notification) map[string]*Notification {
+	m := map[string]*Notification{}
+	for _, n := range ns {
+		m[n.Name] = toNotificationFromProto(n)
+	}
+	return m
+}
+
 // DeleteNotification deletes the notification with the given ID.
 func (b *BucketHandle) DeleteNotification(ctx context.Context, id string) (err error) {
 	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.Bucket.DeleteNotification")
diff --git a/vendor/cloud.google.com/go/storage/reader.go b/vendor/cloud.google.com/go/storage/reader.go
index ab108b0393..7a28b6745e 100644
--- a/vendor/cloud.google.com/go/storage/reader.go
+++ b/vendor/cloud.google.com/go/storage/reader.go
@@ -29,7 +29,6 @@ import (
 
 	"cloud.google.com/go/internal/trace"
 	"google.golang.org/api/googleapi"
-	storagepb "google.golang.org/genproto/googleapis/storage/v2"
 )
 
 var crc32cTable = crc32.MakeTable(crc32.Castagnoli)
@@ -95,7 +94,7 @@ func (o *ObjectHandle) NewRangeReader(ctx context.Context, offset, length int64)
 	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.Object.NewRangeReader")
 	defer func() { trace.EndSpan(ctx, err) }()
 
-	if o.c.gc != nil {
+	if o.c.tc != nil {
 		return o.newRangeReaderWithGRPC(ctx, offset, length)
 	}
 
@@ -383,16 +382,7 @@ type Reader struct {
 	gotCRC             uint32 // running crc
 	reopen             func(seen int64) (*http.Response, error)
 
-	// The following fields are only for use in the gRPC hybrid client.
-	stream         storagepb.Storage_ReadObjectClient
-	reopenWithGRPC func(seen int64) (*readStreamResponse, context.CancelFunc, error)
-	leftovers      []byte
-	cancelStream   context.CancelFunc
-}
-
-type readStreamResponse struct {
-	stream   storagepb.Storage_ReadObjectClient
-	response *storagepb.ReadObjectResponse
+	reader io.ReadCloser
 }
 
 // Close closes the Reader. It must be called when done reading.
@@ -401,14 +391,18 @@ func (r *Reader) Close() error {
 		return r.body.Close()
 	}
 
-	r.closeStream()
+	// TODO(noahdietz): Complete integration means returning this call's return
+	// value, which for gRPC will always be nil.
+	if r.reader != nil {
+		return r.reader.Close()
+	}
 	return nil
 }
 
 func (r *Reader) Read(p []byte) (int, error) {
 	read := r.readWithRetry
-	if r.reopenWithGRPC != nil {
-		read = r.readWithGRPC
+	if r.reader != nil {
+		read = r.reader.Read
 	}
 
 	n, err := read(p)
@@ -438,129 +432,23 @@ func (o *ObjectHandle) newRangeReaderWithGRPC(ctx context.Context, offset, lengt
 	ctx = trace.StartSpan(ctx, "cloud.google.com/go/storage.Object.newRangeReaderWithGRPC")
 	defer func() { trace.EndSpan(ctx, err) }()
 
-	if o.c.gc == nil {
-		err = fmt.Errorf("handle doesn't have a gRPC client initialized")
-		return
-	}
 	if err = o.validate(); err != nil {
 		return
 	}
 
-	// A negative length means "read to the end of the object", but the
-	// read_limit field it corresponds to uses zero to mean the same thing. Thus
-	// we coerce the length to 0 to read to the end of the object.
-	if length < 0 {
-		length = 0
+	params := &newRangeReaderParams{
+		bucket:        o.bucket,
+		object:        o.object,
+		gen:           o.gen,
+		offset:        offset,
+		length:        length,
+		encryptionKey: o.encryptionKey,
+		conds:         o.conds,
 	}
 
-	// For now, there are only globally unique buckets, and "_" is the alias
-	// project ID for such buckets.
-	b := bucketResourceName("_", o.bucket)
-	req := &storagepb.ReadObjectRequest{
-		Bucket: b,
-		Object: o.object,
-	}
-	// The default is a negative value, which means latest.
-	if o.gen >= 0 {
-		req.Generation = o.gen
-	}
+	r, err = o.c.tc.NewRangeReader(ctx, params)
 
-	// Define a function that initiates a Read with offset and length, assuming
-	// we have already read seen bytes.
-	reopen := func(seen int64) (*readStreamResponse, context.CancelFunc, error) {
-		// If the context has already expired, return immediately without making
-		// we call.
-		if err := ctx.Err(); err != nil {
-			return nil, nil, err
-		}
-
-		cc, cancel := context.WithCancel(ctx)
-
-		start := offset + seen
-		// Only set a ReadLimit if length is greater than zero, because zero
-		// means read it all.
-		if length > 0 {
-			req.ReadLimit = length - seen
-		}
-		req.ReadOffset = start
-
-		if err := applyCondsProto("reopenWithGRPC", o.gen, o.conds, req); err != nil {
-			cancel()
-			return nil, nil, err
-		}
-
-		var stream storagepb.Storage_ReadObjectClient
-		var msg *storagepb.ReadObjectResponse
-		var err error
-
-		err = run(cc, func() error {
-			stream, err = o.c.gc.ReadObject(cc, req)
-			if err != nil {
-				return err
-			}
-
-			msg, err = stream.Recv()
-
-			return err
-		}, o.retry, true, setRetryHeaderGRPC(ctx))
-		if err != nil {
-			// Close the stream context we just created to ensure we don't leak
-			// resources.
-			cancel()
-			return nil, nil, err
-		}
-
-		return &readStreamResponse{stream, msg}, cancel, nil
-	}
-
-	res, cancel, err := reopen(0)
-	if err != nil {
-		return nil, err
-	}
-
-	r = &Reader{
-		stream:         res.stream,
-		reopenWithGRPC: reopen,
-		cancelStream:   cancel,
-	}
-
-	// The first message was Recv'd on stream open, use it to populate the
-	// object metadata.
-	msg := res.response
-	obj := msg.GetMetadata()
-	// This is the size of the entire object, even if only a range was requested.
-	size := obj.GetSize()
-
-	r.Attrs = ReaderObjectAttrs{
-		Size:            size,
-		ContentType:     obj.GetContentType(),
-		ContentEncoding: obj.GetContentEncoding(),
-		CacheControl:    obj.GetCacheControl(),
-		LastModified:    obj.GetUpdateTime().AsTime(),
-		Metageneration:  obj.GetMetageneration(),
-		Generation:      obj.GetGeneration(),
-	}
-
-	r.size = size
-	cr := msg.GetContentRange()
-	if cr != nil {
-		r.Attrs.StartOffset = cr.GetStart()
-		r.remain = cr.GetEnd() - cr.GetStart() + 1
-	} else {
-		r.remain = size
-	}
-
-	// Only support checksums when reading an entire object, not a range.
-	if checksums := msg.GetObjectChecksums(); checksums != nil && checksums.Crc32C != nil && offset == 0 && length == 0 {
-		r.wantCRC = checksums.GetCrc32C()
-		r.checkCRC = true
-	}
-
-	// Store the content from the first Recv in the client buffer for reading
-	// later.
-	r.leftovers = msg.GetChecksummedData().GetContent()
-
-	return r, nil
+	return r, err
 }
 
 func (r *Reader) readWithRetry(p []byte) (int, error) {
@@ -586,112 +474,6 @@ func (r *Reader) readWithRetry(p []byte) (int, error) {
 	return n, nil
 }
 
-// closeStream cancels a stream's context in order for it to be closed and
-// collected.
-//
-// This is an experimental API and not intended for public use.
-func (r *Reader) closeStream() {
-	if r.cancelStream != nil {
-		r.cancelStream()
-	}
-	r.stream = nil
-}
-
-// readWithGRPC reads bytes into the user's buffer from an open gRPC stream.
-//
-// This is an experimental API and not intended for public use.
-func (r *Reader) readWithGRPC(p []byte) (int, error) {
-	// No stream to read from, either never initiliazed or Close was called.
-	// Note: There is a potential concurrency issue if multiple routines are
-	// using the same reader. One encounters an error and the stream is closed
-	// and then reopened while the other routine attempts to read from it.
-	if r.stream == nil {
-		return 0, fmt.Errorf("reader has been closed")
-	}
-
-	// The entire object has been read by this reader, return EOF.
-	if r.size != 0 && r.size == r.seen {
-		return 0, io.EOF
-	}
-
-	var n int
-	// Read leftovers and return what was available to conform to the Reader
-	// interface: https://pkg.go.dev/io#Reader.
-	if len(r.leftovers) > 0 {
-		n = copy(p, r.leftovers)
-		r.seen += int64(n)
-		r.leftovers = r.leftovers[n:]
-		return n, nil
-	}
-
-	// Attempt to Recv the next message on the stream.
-	msg, err := r.recv()
-	if err != nil {
-		return 0, err
-	}
-
-	// TODO: Determine if we need to capture incremental CRC32C for this
-	// chunk. The Object CRC32C checksum is captured when directed to read
-	// the entire Object. If directed to read a range, we may need to
-	// calculate the range's checksum for verification if the checksum is
-	// present in the response here.
-	// TODO: Figure out if we need to support decompressive transcoding
-	// https://cloud.google.com/storage/docs/transcoding.
-	content := msg.GetChecksummedData().GetContent()
-	n = copy(p[n:], content)
-	leftover := len(content) - n
-	if leftover > 0 {
-		// Wasn't able to copy all of the data in the message, store for
-		// future Read calls.
-		r.leftovers = content[n:]
-	}
-	r.seen += int64(n)
-
-	return n, nil
-}
-
-// recv attempts to Recv the next message on the stream. In the event
-// that a retryable error is encountered, the stream will be closed, reopened,
-// and Recv again. This will attempt to Recv until one of the following is true:
-//
-// * Recv is successful
-// * A non-retryable error is encountered
-// * The Reader's context is canceled
-//
-// The last error received is the one that is returned, which could be from
-// an attempt to reopen the stream.
-//
-// This is an experimental API and not intended for public use.
-func (r *Reader) recv() (*storagepb.ReadObjectResponse, error) {
-	msg, err := r.stream.Recv()
-	if err != nil && shouldRetry(err) {
-		// This will "close" the existing stream and immediately attempt to
-		// reopen the stream, but will backoff if further attempts are necessary.
-		// Reopening the stream Recvs the first message, so if retrying is
-		// successful, the next logical chunk will be returned.
-		msg, err = r.reopenStream(r.seen)
-	}
-
-	return msg, err
-}
-
-// reopenStream "closes" the existing stream and attempts to reopen a stream and
-// sets the Reader's stream and cancelStream properties in the process.
-//
-// This is an experimental API and not intended for public use.
-func (r *Reader) reopenStream(seen int64) (*storagepb.ReadObjectResponse, error) {
-	// Close existing stream and initialize new stream with updated offset.
-	r.closeStream()
-
-	res, cancel, err := r.reopenWithGRPC(r.seen)
-	if err != nil {
-		return nil, err
-	}
-	r.stream = res.stream
-	r.cancelStream = cancel
-	return res.response, nil
-}
-
 // Size returns the size of the object in bytes.
 // The returned value is always the same and is not affected by
 // calls to Read or Close.
diff --git a/vendor/cloud.google.com/go/storage/storage.go b/vendor/cloud.google.com/go/storage/storage.go
index 719f39723b..e5f7d47340 100644
--- a/vendor/cloud.google.com/go/storage/storage.go
+++ b/vendor/cloud.google.com/go/storage/storage.go
@@ -40,7 +40,6 @@ import (
 	"cloud.google.com/go/internal/optional"
 	"cloud.google.com/go/internal/trace"
 	"cloud.google.com/go/storage/internal"
-	gapic "cloud.google.com/go/storage/internal/apiv2"
 	"github.com/googleapis/gax-go/v2"
 	"golang.org/x/oauth2/google"
 	"google.golang.org/api/googleapi"
@@ -84,6 +83,14 @@ const (
 	// ScopeReadWrite grants permissions to manage your
 	// data in Google Cloud Storage.
 	ScopeReadWrite = raw.DevstorageReadWriteScope
+
+	// aes256Algorithm is the AES256 encryption algorithm used with the
+	// Customer-Supplied Encryption Keys feature.
+	aes256Algorithm = "AES256"
+
+	// defaultGen indicates the latest object generation by default,
+	// using a negative value.
+	defaultGen = int64(-1)
 )
 
 // TODO: remove this once header with invocation ID is applied to all methods.
@@ -106,10 +113,8 @@ type Client struct {
 	creds *google.Credentials
 	retry *retryConfig
 
-	// gc is an optional gRPC-based, GAPIC client.
-	//
-	// This is an experimental field and not intended for public use.
-	gc *gapic.Client
+	// tc is the transport-agnostic client implemented with either gRPC or HTTP.
+	tc storageClient
 }
 
 // NewClient creates a new Google Cloud Storage client.
@@ -205,12 +210,12 @@ func NewClient(ctx context.Context, opts ...option.ClientOption) (*Client, error
 // This is an experimental API and not intended for public use.
 func newGRPCClient(ctx context.Context, opts ...option.ClientOption) (*Client, error) {
 	opts = append(defaultGRPCOptions(), opts...)
-	g, err := gapic.NewClient(ctx, opts...)
+	tc, err := newGRPCStorageClient(ctx, withClientOptions(opts...))
 	if err != nil {
 		return nil, err
 	}
 
-	return &Client{gc: g}, nil
+	return &Client{tc: tc}, nil
 }
 
 // Close closes the Client.
@@ -221,8 +226,8 @@ func (c *Client) Close() error {
 	c.hc = nil
 	c.raw = nil
 	c.creds = nil
-	if c.gc != nil {
-		return c.gc.Close()
+	if c.tc != nil {
+		return c.tc.Close()
 	}
 	return nil
 }
@@ -1231,6 +1236,49 @@ func (o *ObjectAttrs) toProtoObject(b string) *storagepb.Object {
 	}
 }
 
+// toProtoObject copies the attributes to update from uattrs to the proto library's Object type.
+func (uattrs *ObjectAttrsToUpdate) toProtoObject(bucket, object string) *storagepb.Object {
+	o := &storagepb.Object{
+		Name:   object,
+		Bucket: bucket,
+	}
+	if uattrs == nil {
+		return o
+	}
+
+	if uattrs.EventBasedHold != nil {
+		o.EventBasedHold = proto.Bool(optional.ToBool(uattrs.EventBasedHold))
+	}
+	if uattrs.TemporaryHold != nil {
+		o.TemporaryHold = optional.ToBool(uattrs.TemporaryHold)
+	}
+	if uattrs.ContentType != nil {
+		o.ContentType = optional.ToString(uattrs.ContentType)
+	}
+	if uattrs.ContentLanguage != nil {
+		o.ContentLanguage = optional.ToString(uattrs.ContentLanguage)
+	}
+	if uattrs.ContentEncoding != nil {
+		o.ContentEncoding = optional.ToString(uattrs.ContentEncoding)
+	}
+	if uattrs.ContentDisposition != nil {
+		o.ContentDisposition = optional.ToString(uattrs.ContentDisposition)
+	}
+	if uattrs.CacheControl != nil {
+		o.CacheControl = optional.ToString(uattrs.CacheControl)
+	}
+	if !uattrs.CustomTime.IsZero() {
+		o.CustomTime = toProtoTimestamp(uattrs.CustomTime)
+	}
+	if uattrs.ACL != nil {
+		o.Acl = toProtoObjectACL(uattrs.ACL)
+	}
+
+	// TODO(cathyo): Handle metadata. Pending b/230510191.
+
+	return o
+}
+
 // ObjectAttrs represents the metadata for a Google Cloud Storage (GCS) object.
 type ObjectAttrs struct {
 	// Bucket is the name of the bucket containing this GCS object.
@@ -1312,6 +1360,10 @@ type ObjectAttrs struct {
 
 	// Metadata represents user-provided metadata, in key/value pairs.
 	// It can be nil if no metadata is provided.
+	//
+	// For object downloads using Reader, metadata keys are sent as headers.
+	// Therefore, avoid setting metadata keys using characters that are not valid
+	// for headers. See https://www.rfc-editor.org/rfc/rfc7230#section-3.2.6.
 	Metadata map[string]string
 
 	// Generation is the generation number of the object's content.
@@ -1992,13 +2044,26 @@ func setEncryptionHeaders(headers http.Header, key []byte, copySource bool) erro
 	if copySource {
 		cs = "copy-source-"
 	}
-	headers.Set("x-goog-"+cs+"encryption-algorithm", "AES256")
+	headers.Set("x-goog-"+cs+"encryption-algorithm", aes256Algorithm)
 	headers.Set("x-goog-"+cs+"encryption-key", base64.StdEncoding.EncodeToString(key))
 	keyHash := sha256.Sum256(key)
 	headers.Set("x-goog-"+cs+"encryption-key-sha256", base64.StdEncoding.EncodeToString(keyHash[:]))
 	return nil
 }
 
+// toProtoCommonObjectRequestParams sets customer-supplied encryption to the proto library's CommonObjectRequestParams.
+func toProtoCommonObjectRequestParams(key []byte) *storagepb.CommonObjectRequestParams {
+	if key == nil {
+		return nil
+	}
+	keyHash := sha256.Sum256(key)
+	return &storagepb.CommonObjectRequestParams{
+		EncryptionAlgorithm:      aes256Algorithm,
+		EncryptionKeyBytes:       key,
+		EncryptionKeySha256Bytes: keyHash[:],
+	}
+}
+
 // ServiceAccount fetches the email address of the given project's Google Cloud Storage service account.
 func (c *Client) ServiceAccount(ctx context.Context, projectID string) (string, error) {
 	r := c.raw.Projects.ServiceAccount.Get(projectID)
diff --git a/vendor/cloud.google.com/go/storage/writer.go b/vendor/cloud.google.com/go/storage/writer.go
index c616603e4c..1908826e63 100644
--- a/vendor/cloud.google.com/go/storage/writer.go
+++ b/vendor/cloud.google.com/go/storage/writer.go
@@ -24,12 +24,9 @@ import (
 	"time"
 	"unicode/utf8"
 
-	"github.com/golang/protobuf/proto"
 	"google.golang.org/api/googleapi"
 	raw "google.golang.org/api/storage/v1"
 	storagepb "google.golang.org/genproto/googleapis/storage/v2"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
 )
 
 const (
@@ -123,16 +120,6 @@ type Writer struct {
 
 	mu  sync.Mutex
 	err error
-
-	// The gRPC client-stream used for sending buffers.
-	//
-	// This is an experimental API and not intended for public use.
-	stream storagepb.Storage_WriteObjectClient
-
-	// The Resumable Upload ID started by a gRPC-based Writer.
-	//
-	// This is an experimental API and not intended for public use.
-	upid string
 }
 
 func (w *Writer) open() error {
@@ -249,8 +236,8 @@ func (w *Writer) Write(p []byte) (n int, err error) {
 	}
 	if !w.opened {
 		// gRPC client has been initialized - use gRPC to upload.
-		if w.o.c.gc != nil {
-			if err := w.openGRPC(); err != nil {
+		if w.o.c.tc != nil {
+			if err := w.openWriter(); err != nil {
 				return 0, err
 			}
 		} else if err := w.open(); err != nil {
@@ -277,7 +264,11 @@ func (w *Writer) Write(p []byte) (n int, err error) {
 // can be retrieved by calling Attrs.
 func (w *Writer) Close() error {
 	if !w.opened {
-		if err := w.open(); err != nil {
+		if w.o.c.tc != nil {
+			if err := w.openWriter(); err != nil {
+				return err
+			}
+		} else if err := w.open(); err != nil {
 			return err
 		}
 	}
@@ -293,6 +284,35 @@ func (w *Writer) Close() error {
 	return w.err
 }
 
+func (w *Writer) openWriter() (err error) {
+	if err := w.validateWriteAttrs(); err != nil {
+		return err
+	}
+
+	go w.monitorCancel()
+	params := &openWriterParams{
+		ctx:                w.ctx,
+		chunkSize:          w.ChunkSize,
+		chunkRetryDeadline: w.ChunkRetryDeadline,
+		bucket:             w.o.bucket,
+		attrs:              &w.ObjectAttrs,
+		conds:              w.o.conds,
+		encryptionKey:      w.o.encryptionKey,
+		sendCRC32C:         w.SendCRC32C,
+		donec:              w.donec,
+		setError:           w.error,
+		progress:           w.progress,
+		setObj:             func(o *ObjectAttrs) { w.obj = o },
+	}
+	w.pw, err = w.o.c.tc.OpenWriter(params)
+	if err != nil {
+		return err
+	}
+	w.opened = true
+
+	return nil
+}
+
 // monitorCancel is intended to be used as a background goroutine. It monitors the
 // context, and when it observes that the context has been canceled, it manually
 // closes things that do not take a context.
@@ -361,333 +381,3 @@ func (w *Writer) error(err error) {
 	w.err = err
 	w.mu.Unlock()
 }
-
-// openGRPC initializes a pipe for the user to write data to, and a routine to
-// read from that pipe and upload the data to GCS via gRPC.
-//
-// This is an experimental API and not intended for public use.
-func (w *Writer) openGRPC() error {
-	if err := w.validateWriteAttrs(); err != nil {
-		return err
-	}
-
-	pr, pw := io.Pipe()
-	w.pw = pw
-	w.opened = true
-
-	go w.monitorCancel()
-
-	bufSize := w.ChunkSize
-	if w.ChunkSize == 0 {
-		// TODO: Should we actually use the minimum of 256 KB here when the user
-		// indicates they want minimal memory usage? We cannot do a zero-copy,
-		// bufferless upload like HTTP/JSON can.
-		// TODO: We need to determine if we can avoid starting a
-		// resumable upload when the user *plans* to send more than bufSize but
-		// with a bufferless upload.
-		bufSize = maxPerMessageWriteSize
-	}
-	buf := make([]byte, bufSize)
-
-	var offset int64
-
-	// This function reads the data sent to the pipe and sends sets of messages
-	// on the gRPC client-stream as the buffer is filled.
-	go func() {
-		defer close(w.donec)
-
-		// Loop until there is an error or the Object has been finalized.
-		for {
-			// Note: This blocks until either the buffer is full or EOF is read.
-			recvd, doneReading, err := read(pr, buf)
-			if err != nil {
-				err = checkCanceled(err)
-				w.error(err)
-				pr.CloseWithError(err)
-				return
-			}
-			toWrite := buf[:recvd]
-
-			// TODO: Figure out how to set up encryption via CommonObjectRequestParams.
-
-			// The chunk buffer is full, but there is no end in sight. This
-			// means that a resumable upload will need to be used to send
-			// multiple chunks, until we are done reading data. Start a
-			// resumable upload if it has not already been started.
-			// Otherwise, all data will be sent over a single gRPC stream.
-			if !doneReading && w.upid == "" {
-				err = w.startResumableUpload()
-				if err != nil {
-					err = checkCanceled(err)
-					w.error(err)
-					pr.CloseWithError(err)
-					return
-				}
-			}
-
-			o, off, finalized, err := w.uploadBuffer(toWrite, recvd, offset, doneReading)
-			if err != nil {
-				err = checkCanceled(err)
-				w.error(err)
-				pr.CloseWithError(err)
-				return
-			}
-			// At this point, the current buffer has been uploaded. Capture the
-			// committed offset here in case the upload was not finalized and
-			// another chunk is to be uploaded.
-			offset = off
-			w.progress(offset)
-
-			// When we are done reading data and the chunk has been finalized,
-			// we are done.
-			if doneReading && finalized {
-				// Build Object from server's response.
-				w.obj = newObjectFromProto(o)
-				return
-			}
-		}
-	}()
-
-	return nil
-}
-
-// startResumableUpload initializes a Resumable Upload with gRPC and sets the
-// upload ID on the Writer.
-//
-// This is an experimental API and not intended for public use.
-func (w *Writer) startResumableUpload() error {
-	spec, err := w.writeObjectSpec()
-	if err != nil {
-		return err
-	}
-	upres, err := w.o.c.gc.StartResumableWrite(w.ctx, &storagepb.StartResumableWriteRequest{
-		WriteObjectSpec: spec,
-	})
-
-	w.upid = upres.GetUploadId()
-	return err
-}
-
-// queryProgress is a helper that queries the status of the resumable upload
-// associated with the given upload ID.
-//
-// This is an experimental API and not intended for public use.
-func (w *Writer) queryProgress() (int64, error) {
-	q, err := w.o.c.gc.QueryWriteStatus(w.ctx, &storagepb.QueryWriteStatusRequest{UploadId: w.upid})
-
-	// q.GetCommittedSize() will return 0 if q is nil.
-	return q.GetPersistedSize(), err
-}
-
-// uploadBuffer opens a Write stream and uploads the buffer at the given offset (if
-// uploading a chunk for a resumable uploadBuffer), and will mark the write as
-// finished if we are done receiving data from the user. The resulting write
-// offset after uploading the buffer is returned, as well as a boolean
-// indicating if the Object has been finalized. If it has been finalized, the
-// final Object will be returned as well. Finalizing the upload is primarily
-// important for Resumable Uploads. A simple or multi-part upload will always
-// be finalized once the entire buffer has been written.
-//
-// This is an experimental API and not intended for public use.
-func (w *Writer) uploadBuffer(buf []byte, recvd int, start int64, doneReading bool) (*storagepb.Object, int64, bool, error) {
-	var err error
-	var finishWrite bool
-	var sent, limit int = 0, maxPerMessageWriteSize
-	offset := start
-	for {
-		first := sent == 0
-		// This indicates that this is the last message and the remaining
-		// data fits in one message.
-		belowLimit := recvd-sent <= limit
-		if belowLimit {
-			limit = recvd - sent
-		}
-		if belowLimit && doneReading {
-			finishWrite = true
-		}
-
-		// Prepare chunk section for upload.
-		data := buf[sent : sent+limit]
-		req := &storagepb.WriteObjectRequest{
-			Data: &storagepb.WriteObjectRequest_ChecksummedData{
-				ChecksummedData: &storagepb.ChecksummedData{
-					Content: data,
-				},
-			},
-			WriteOffset: offset,
-			FinishWrite: finishWrite,
-		}
-
-		// Open a new stream and set the first_message field on the request.
-		// The first message on the WriteObject stream must either be the
-		// Object or the Resumable Upload ID.
-		if first {
-			w.stream, err = w.o.c.gc.WriteObject(w.ctx)
-			if err != nil {
-				return nil, 0, false, err
-			}
-
-			if w.upid != "" {
-				req.FirstMessage = &storagepb.WriteObjectRequest_UploadId{UploadId: w.upid}
-			} else {
-				spec, err := w.writeObjectSpec()
-				if err != nil {
-					return nil, 0, false, err
-				}
-				req.FirstMessage = &storagepb.WriteObjectRequest_WriteObjectSpec{
-					WriteObjectSpec: spec,
-				}
-			}
-
-			// TODO: Currently the checksums are only sent on the first message
-			// of the stream, but in the future, we must also support sending it
-			// on the *last* message of the stream (instead of the first).
-			if w.SendCRC32C {
-				req.ObjectChecksums = &storagepb.ObjectChecksums{
-					Crc32C:  proto.Uint32(w.CRC32C),
-					Md5Hash: w.MD5,
-				}
-			}
-		}
-
-		err = w.stream.Send(req)
-		if err == io.EOF {
-			// err was io.EOF. The client-side of a stream only gets an EOF on Send
-			// when the backend closes the stream and wants to return an error
-			// status. Closing the stream receives the status as an error.
-			_, err = w.stream.CloseAndRecv()
-
-			// Retriable errors mean we should start over and attempt to
-			// resend the entire buffer via a new stream.
-			// If not retriable, falling through will return the error received
-			// from closing the stream.
-			if shouldRetry(err) {
-				sent = 0
-				finishWrite = false
-				// TODO: Add test case for failure modes of querying progress.
-				offset, err = w.determineOffset(start)
-				if err == nil {
-					continue
-				}
-			}
-		}
-		if err != nil {
-			return nil, 0, false, err
-		}
-
-		// Update the immediate stream's sent total and the upload offset with
-		// the data sent.
-		sent += len(data)
-		offset += int64(len(data))
-
-		// Not done sending data, do not attempt to commit it yet, loop around
-		// and send more data.
-		if recvd-sent > 0 {
-			continue
-		}
-
-		// Done sending data. Close the stream to "commit" the data sent.
-		resp, finalized, err := w.commit()
-		// Retriable errors mean we should start over and attempt to
-		// resend the entire buffer via a new stream.
-		// If not retriable, falling through will return the error received
-		// from closing the stream.
-		if shouldRetry(err) {
-			sent = 0
-			finishWrite = false
-			offset, err = w.determineOffset(start)
-			if err == nil {
-				continue
-			}
-		}
-		if err != nil {
-			return nil, 0, false, err
-		}
-
-		return resp.GetResource(), offset, finalized, nil
-	}
-}
-
-// determineOffset either returns the offset given to it in the case of a simple
-// upload, or queries the write status in the case a resumable upload is being
-// used.
-//
-// This is an experimental API and not intended for public use.
-func (w *Writer) determineOffset(offset int64) (int64, error) {
-	// For a Resumable Upload, we must start from however much data
-	// was committed.
-	if w.upid != "" {
-		committed, err := w.queryProgress()
-		if err != nil {
-			return 0, err
-		}
-		offset = committed
-	}
-	return offset, nil
-}
-
-// commit closes the stream to commit the data sent and potentially receive
-// the finalized object if finished uploading. If the last request sent
-// indicated that writing was finished, the Object will be finalized and
-// returned. If not, then the Object will be nil, and the boolean returned will
-// be false.
-//
-// This is an experimental API and not intended for public use.
-func (w *Writer) commit() (*storagepb.WriteObjectResponse, bool, error) {
-	finalized := true
-	resp, err := w.stream.CloseAndRecv()
-	if err == io.EOF {
-		// Closing a stream for a resumable upload finish_write = false results
-		// in an EOF which can be ignored, as we aren't done uploading yet.
-		finalized = false
-		err = nil
-	}
-	// Drop the stream reference as it has been closed.
-	w.stream = nil
-
-	return resp, finalized, err
-}
-
-// writeObjectSpec constructs a WriteObjectSpec proto using the Writer's
-// ObjectAttrs and applies its Conditions. This is only used for gRPC.
-//
-// This is an experimental API and not intended for public use.
-func (w *Writer) writeObjectSpec() (*storagepb.WriteObjectSpec, error) {
-	spec := &storagepb.WriteObjectSpec{
-		Resource: w.ObjectAttrs.toProtoObject(w.o.bucket),
-	}
-	// WriteObject doesn't support the generation condition, so use -1.
-	if err := applyCondsProto("WriteObject", -1, w.o.conds, spec); err != nil {
-		return nil, err
-	}
-	return spec, nil
-}
-
-// read copies the data in the reader to the given buffer and reports how much
-// data was read into the buffer and if there is no more data to read (EOF).
-//
-// This is an experimental API and not intended for public use.
-func read(r io.Reader, buf []byte) (int, bool, error) {
-	// Set n to -1 to start the Read loop.
-	var n, recvd int = -1, 0
-	var err error
-	for err == nil && n != 0 {
-		// The routine blocks here until data is received.
-		n, err = r.Read(buf[recvd:])
-		recvd += n
-	}
-	var done bool
-	if err == io.EOF {
-		done = true
-		err = nil
-	}
-	return recvd, done, err
-}
-
-func checkCanceled(err error) error {
-	if status.Code(err) == codes.Canceled {
-		return context.Canceled
-	}
-
-	return err
-}
diff --git a/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go b/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go
index 4a1a0f9ef8..9deff77197 100644
--- a/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go
+++ b/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go
@@ -16792,43 +16792,6 @@ var awsPartition = partition{
 				},
 			},
 		},
-		"redshift-serverless": service{
-			Endpoints: serviceEndpoints{
-				endpointKey{
-					Region: "ap-northeast-1",
-				}: endpoint{},
-				endpointKey{
-					Region: "ap-northeast-2",
-				}: endpoint{},
-				endpointKey{
-					Region: "ap-southeast-1",
-				}: endpoint{},
-				endpointKey{
-					Region: "ap-southeast-2",
-				}: endpoint{},
-				endpointKey{
-					Region: "eu-central-1",
-				}: endpoint{},
-				endpointKey{
-					Region: "eu-north-1",
-				}: endpoint{},
-				endpointKey{
-					Region: "eu-west-1",
-				}: endpoint{},
-				endpointKey{
-					Region: "eu-west-2",
-				}: endpoint{},
-				endpointKey{
-					Region: "us-east-1",
-				}: endpoint{},
-				endpointKey{
-					Region: "us-east-2",
-				}: endpoint{},
-				endpointKey{
-					Region: "us-west-2",
-				}: endpoint{},
-			},
-		},
 		"rekognition": service{
 			Endpoints: serviceEndpoints{
 				endpointKey{
diff --git a/vendor/github.com/aws/aws-sdk-go/aws/version.go b/vendor/github.com/aws/aws-sdk-go/aws/version.go
index 3bf33f8de1..e6612ead48 100644
--- a/vendor/github.com/aws/aws-sdk-go/aws/version.go
+++ b/vendor/github.com/aws/aws-sdk-go/aws/version.go
@@ -5,4 +5,4 @@ package aws
 const SDKName = "aws-sdk-go"
 
 // SDKVersion is the version of this SDK
-const SDKVersion = "1.44.37"
+const SDKVersion = "1.44.43"
diff --git a/vendor/golang.org/x/oauth2/authhandler/authhandler.go b/vendor/golang.org/x/oauth2/authhandler/authhandler.go
index 69967cf87e..9bc6cd7bc5 100644
--- a/vendor/golang.org/x/oauth2/authhandler/authhandler.go
+++ b/vendor/golang.org/x/oauth2/authhandler/authhandler.go
@@ -13,11 +13,36 @@ import (
 	"golang.org/x/oauth2"
 )
 
+const (
+	// Parameter keys for AuthCodeURL method to support PKCE.
+	codeChallengeKey       = "code_challenge"
+	codeChallengeMethodKey = "code_challenge_method"
+
+	// Parameter key for Exchange method to support PKCE.
+	codeVerifierKey = "code_verifier"
+)
+
+// PKCEParams holds parameters to support PKCE.
+type PKCEParams struct {
+	Challenge       string // The unpadded, base64-url-encoded string of the encrypted code verifier.
+	ChallengeMethod string // The encryption method (ex. S256).
+	Verifier        string // The original, non-encrypted secret.
+}
+
 // AuthorizationHandler is a 3-legged-OAuth helper that prompts
 // the user for OAuth consent at the specified auth code URL
 // and returns an auth code and state upon approval.
 type AuthorizationHandler func(authCodeURL string) (code string, state string, err error)
 
+// TokenSourceWithPKCE is an enhanced version of TokenSource with PKCE support.
+//
+// The pkce parameter supports PKCE flow, which uses code challenge and code verifier
+// to prevent CSRF attacks. A unique code challenge and code verifier should be generated
+// by the caller at runtime. See https://www.oauth.com/oauth2-servers/pkce/ for more info.
+func TokenSourceWithPKCE(ctx context.Context, config *oauth2.Config, state string, authHandler AuthorizationHandler, pkce *PKCEParams) oauth2.TokenSource {
+	return oauth2.ReuseTokenSource(nil, authHandlerSource{config: config, ctx: ctx, authHandler: authHandler, state: state, pkce: pkce})
+}
+
 // TokenSource returns an oauth2.TokenSource that fetches access tokens
 // using 3-legged-OAuth flow.
 //
@@ -33,7 +58,7 @@ type AuthorizationHandler func(authCodeURL string) (code string, state string, e
 // and response before exchanging the auth code for OAuth token to prevent CSRF
 // attacks.
 func TokenSource(ctx context.Context, config *oauth2.Config, state string, authHandler AuthorizationHandler) oauth2.TokenSource {
-	return oauth2.ReuseTokenSource(nil, authHandlerSource{config: config, ctx: ctx, authHandler: authHandler, state: state})
+	return TokenSourceWithPKCE(ctx, config, state, authHandler, nil)
 }
 
 type authHandlerSource struct {
@@ -41,10 +66,17 @@ type authHandlerSource struct {
 	config      *oauth2.Config
 	authHandler AuthorizationHandler
 	state       string
+	pkce        *PKCEParams
 }
 
 func (source authHandlerSource) Token() (*oauth2.Token, error) {
-	url := source.config.AuthCodeURL(source.state)
+	// Step 1: Obtain auth code.
+	var authCodeUrlOptions []oauth2.AuthCodeOption
+	if source.pkce != nil && source.pkce.Challenge != "" && source.pkce.ChallengeMethod != "" {
+		authCodeUrlOptions = []oauth2.AuthCodeOption{oauth2.SetAuthURLParam(codeChallengeKey, source.pkce.Challenge),
+			oauth2.SetAuthURLParam(codeChallengeMethodKey, source.pkce.ChallengeMethod)}
+	}
+	url := source.config.AuthCodeURL(source.state, authCodeUrlOptions...)
 	code, state, err := source.authHandler(url)
 	if err != nil {
 		return nil, err
@@ -52,5 +84,11 @@ func (source authHandlerSource) Token() (*oauth2.Token, error) {
 	if state != source.state {
 		return nil, errors.New("state mismatch in 3-legged-OAuth flow")
 	}
-	return source.config.Exchange(source.ctx, code)
+
+	// Step 2: Exchange auth code for access token.
+	var exchangeOptions []oauth2.AuthCodeOption
+	if source.pkce != nil && source.pkce.Verifier != "" {
+		exchangeOptions = []oauth2.AuthCodeOption{oauth2.SetAuthURLParam(codeVerifierKey, source.pkce.Verifier)}
+	}
+	return source.config.Exchange(source.ctx, code, exchangeOptions...)
 }
diff --git a/vendor/golang.org/x/oauth2/google/default.go b/vendor/golang.org/x/oauth2/google/default.go
index 024a104b0d..7ed02cd414 100644
--- a/vendor/golang.org/x/oauth2/google/default.go
+++ b/vendor/golang.org/x/oauth2/google/default.go
@@ -54,11 +54,14 @@ type CredentialsParams struct {
 	// Optional.
 	Subject string
 
-	// AuthHandler is the AuthorizationHandler used for 3-legged OAuth flow. Optional.
+	// AuthHandler is the AuthorizationHandler used for 3-legged OAuth flow. Required for 3LO flow.
 	AuthHandler authhandler.AuthorizationHandler
 
-	// State is a unique string used with AuthHandler. Optional.
+	// State is a unique string used with AuthHandler. Required for 3LO flow.
 	State string
+
+	// PKCE is used to support PKCE flow. Optional for 3LO flow.
+	PKCE *authhandler.PKCEParams
 }
 
 func (params CredentialsParams) deepCopy() CredentialsParams {
@@ -176,7 +179,7 @@ func CredentialsFromJSONWithParams(ctx context.Context, jsonData []byte, params
 	if config != nil {
 		return &Credentials{
 			ProjectID:   "",
-			TokenSource: authhandler.TokenSource(ctx, config, params.State, params.AuthHandler),
+			TokenSource: authhandler.TokenSourceWithPKCE(ctx, config, params.State, params.AuthHandler, params.PKCE),
 			JSON:        jsonData,
 		}, nil
 	}
diff --git a/vendor/golang.org/x/sys/unix/syscall_aix.go b/vendor/golang.org/x/sys/unix/syscall_aix.go
index ad22c33db3..ac579c60fe 100644
--- a/vendor/golang.org/x/sys/unix/syscall_aix.go
+++ b/vendor/golang.org/x/sys/unix/syscall_aix.go
@@ -217,12 +217,12 @@ func Accept(fd int) (nfd int, sa Sockaddr, err error) {
 	return
 }
 
-func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
+func recvmsgRaw(fd int, iov []Iovec, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
 	// Recvmsg not implemented on AIX
 	return -1, -1, -1, ENOSYS
 }
 
-func sendmsgN(fd int, p, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
+func sendmsgN(fd int, iov []Iovec, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
 	// SendmsgN not implemented on AIX
 	return -1, ENOSYS
 }
diff --git a/vendor/golang.org/x/sys/unix/syscall_bsd.go b/vendor/golang.org/x/sys/unix/syscall_bsd.go
index 9c87c5f07f..c437fc5d7b 100644
--- a/vendor/golang.org/x/sys/unix/syscall_bsd.go
+++ b/vendor/golang.org/x/sys/unix/syscall_bsd.go
@@ -325,27 +325,26 @@ func GetsockoptString(fd, level, opt int) (string, error) {
 //sys	sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error)
 //sys	recvmsg(s int, msg *Msghdr, flags int) (n int, err error)
 
-func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
+func recvmsgRaw(fd int, iov []Iovec, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
 	var msg Msghdr
 	msg.Name = (*byte)(unsafe.Pointer(rsa))
 	msg.Namelen = uint32(SizeofSockaddrAny)
-	var iov Iovec
-	if len(p) > 0 {
-		iov.Base = (*byte)(unsafe.Pointer(&p[0]))
-		iov.SetLen(len(p))
-	}
 	var dummy byte
 	if len(oob) > 0 {
 		// receive at least one normal byte
-		if len(p) == 0 {
-			iov.Base = &dummy
-			iov.SetLen(1)
+		if emptyIovecs(iov) {
+			var iova [1]Iovec
+			iova[0].Base = &dummy
+			iova[0].SetLen(1)
+			iov = iova[:]
 		}
 		msg.Control = (*byte)(unsafe.Pointer(&oob[0]))
 		msg.SetControllen(len(oob))
 	}
-	msg.Iov = &iov
-	msg.Iovlen = 1
+	if len(iov) > 0 {
+		msg.Iov = &iov[0]
+		msg.SetIovlen(len(iov))
+	}
 	if n, err = recvmsg(fd, &msg, flags); err != nil {
 		return
 	}
@@ -356,31 +355,32 @@ func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn
 
 //sys	sendmsg(s int, msg *Msghdr, flags int) (n int, err error)
 
-func sendmsgN(fd int, p, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
+func sendmsgN(fd int, iov []Iovec, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
 	var msg Msghdr
 	msg.Name = (*byte)(unsafe.Pointer(ptr))
 	msg.Namelen = uint32(salen)
-	var iov Iovec
-	if len(p) > 0 {
-		iov.Base = (*byte)(unsafe.Pointer(&p[0]))
-		iov.SetLen(len(p))
-	}
 	var dummy byte
+	var empty bool
 	if len(oob) > 0 {
 		// send at least one normal byte
-		if len(p) == 0 {
-			iov.Base = &dummy
-			iov.SetLen(1)
+		empty := emptyIovecs(iov)
+		if empty {
+			var iova [1]Iovec
+			iova[0].Base = &dummy
+			iova[0].SetLen(1)
+			iov = iova[:]
 		}
 		msg.Control = (*byte)(unsafe.Pointer(&oob[0]))
 		msg.SetControllen(len(oob))
 	}
-	msg.Iov = &iov
-	msg.Iovlen = 1
+	if len(iov) > 0 {
+		msg.Iov = &iov[0]
+		msg.SetIovlen(len(iov))
+	}
 	if n, err = sendmsg(fd, &msg, flags); err != nil {
 		return 0, err
 	}
-	if len(oob) > 0 && len(p) == 0 {
+	if len(oob) > 0 && empty {
 		n = 0
 	}
 	return n, nil
diff --git a/vendor/golang.org/x/sys/unix/syscall_illumos.go b/vendor/golang.org/x/sys/unix/syscall_illumos.go
index 8d5f294c42..e48244a9c9 100644
--- a/vendor/golang.org/x/sys/unix/syscall_illumos.go
+++ b/vendor/golang.org/x/sys/unix/syscall_illumos.go
@@ -20,10 +20,9 @@ func bytes2iovec(bs [][]byte) []Iovec {
 	for i, b := range bs {
 		iovecs[i].SetLen(len(b))
 		if len(b) > 0 {
-			// somehow Iovec.Base on illumos is (*int8), not (*byte)
-			iovecs[i].Base = (*int8)(unsafe.Pointer(&b[0]))
+			iovecs[i].Base = &b[0]
 		} else {
-			iovecs[i].Base = (*int8)(unsafe.Pointer(&_zero))
+			iovecs[i].Base = (*byte)(unsafe.Pointer(&_zero))
 		}
 	}
 	return iovecs
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux.go b/vendor/golang.org/x/sys/unix/syscall_linux.go
index c8d2032125..5e4a94f731 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux.go
@@ -1499,18 +1499,13 @@ func KeyctlRestrictKeyring(ringid int, keyType string, restriction string) error
 //sys	keyctlRestrictKeyringByType(cmd int, arg2 int, keyType string, restriction string) (err error) = SYS_KEYCTL
 //sys	keyctlRestrictKeyring(cmd int, arg2 int) (err error) = SYS_KEYCTL
 
-func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
+func recvmsgRaw(fd int, iov []Iovec, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
 	var msg Msghdr
 	msg.Name = (*byte)(unsafe.Pointer(rsa))
 	msg.Namelen = uint32(SizeofSockaddrAny)
-	var iov Iovec
-	if len(p) > 0 {
-		iov.Base = &p[0]
-		iov.SetLen(len(p))
-	}
 	var dummy byte
 	if len(oob) > 0 {
-		if len(p) == 0 {
+		if emptyIovecs(iov) {
 			var sockType int
 			sockType, err = GetsockoptInt(fd, SOL_SOCKET, SO_TYPE)
 			if err != nil {
@@ -1518,15 +1513,19 @@ func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn
 			}
 			// receive at least one normal byte
 			if sockType != SOCK_DGRAM {
-				iov.Base = &dummy
-				iov.SetLen(1)
+				var iova [1]Iovec
+				iova[0].Base = &dummy
+				iova[0].SetLen(1)
+				iov = iova[:]
 			}
 		}
 		msg.Control = &oob[0]
 		msg.SetControllen(len(oob))
 	}
-	msg.Iov = &iov
-	msg.Iovlen = 1
+	if len(iov) > 0 {
+		msg.Iov = &iov[0]
+		msg.SetIovlen(len(iov))
+	}
 	if n, err = recvmsg(fd, &msg, flags); err != nil {
 		return
 	}
@@ -1535,18 +1534,15 @@ func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn
 	return
 }
 
-func sendmsgN(fd int, p, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
+func sendmsgN(fd int, iov []Iovec, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
 	var msg Msghdr
 	msg.Name = (*byte)(ptr)
 	msg.Namelen = uint32(salen)
-	var iov Iovec
-	if len(p) > 0 {
-		iov.Base = &p[0]
-		iov.SetLen(len(p))
-	}
 	var dummy byte
+	var empty bool
 	if len(oob) > 0 {
-		if len(p) == 0 {
+		empty := emptyIovecs(iov)
+		if empty {
 			var sockType int
 			sockType, err = GetsockoptInt(fd, SOL_SOCKET, SO_TYPE)
 			if err != nil {
@@ -1554,19 +1550,22 @@ func sendmsgN(fd int, p, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags i
 			}
 			// send at least one normal byte
 			if sockType != SOCK_DGRAM {
-				iov.Base = &dummy
-				iov.SetLen(1)
+				var iova [1]Iovec
+				iova[0].Base = &dummy
+				iova[0].SetLen(1)
 			}
 		}
 		msg.Control = &oob[0]
 		msg.SetControllen(len(oob))
 	}
-	msg.Iov = &iov
-	msg.Iovlen = 1
+	if len(iov) > 0 {
+		msg.Iov = &iov[0]
+		msg.SetIovlen(len(iov))
+	}
 	if n, err = sendmsg(fd, &msg, flags); err != nil {
 		return 0, err
 	}
-	if len(oob) > 0 && len(p) == 0 {
+	if len(oob) > 0 && empty {
 		n = 0
 	}
 	return n, nil
diff --git a/vendor/golang.org/x/sys/unix/syscall_openbsd_mips64.go b/vendor/golang.org/x/sys/unix/syscall_openbsd_mips64.go
index 30f285343e..1378489f8d 100644
--- a/vendor/golang.org/x/sys/unix/syscall_openbsd_mips64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_openbsd_mips64.go
@@ -26,6 +26,10 @@ func (msghdr *Msghdr) SetControllen(length int) {
 	msghdr.Controllen = uint32(length)
 }
 
+func (msghdr *Msghdr) SetIovlen(length int) {
+	msghdr.Iovlen = uint32(length)
+}
+
 func (cmsg *Cmsghdr) SetLen(length int) {
 	cmsg.Len = uint32(length)
 }
diff --git a/vendor/golang.org/x/sys/unix/syscall_solaris.go b/vendor/golang.org/x/sys/unix/syscall_solaris.go
index 932996c75b..b5ec457cdc 100644
--- a/vendor/golang.org/x/sys/unix/syscall_solaris.go
+++ b/vendor/golang.org/x/sys/unix/syscall_solaris.go
@@ -451,26 +451,25 @@ func Accept(fd int) (nfd int, sa Sockaddr, err error) {
 
 //sys	recvmsg(s int, msg *Msghdr, flags int) (n int, err error) = libsocket.__xnet_recvmsg
 
-func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
+func recvmsgRaw(fd int, iov []Iovec, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn int, recvflags int, err error) {
 	var msg Msghdr
 	msg.Name = (*byte)(unsafe.Pointer(rsa))
 	msg.Namelen = uint32(SizeofSockaddrAny)
-	var iov Iovec
-	if len(p) > 0 {
-		iov.Base = (*int8)(unsafe.Pointer(&p[0]))
-		iov.SetLen(len(p))
-	}
-	var dummy int8
+	var dummy byte
 	if len(oob) > 0 {
 		// receive at least one normal byte
-		if len(p) == 0 {
-			iov.Base = &dummy
-			iov.SetLen(1)
+		if emptyIovecs(iov) {
+			var iova [1]Iovec
+			iova[0].Base = &dummy
+			iova[0].SetLen(1)
+			iov = iova[:]
 		}
 		msg.Accrightslen = int32(len(oob))
 	}
-	msg.Iov = &iov
-	msg.Iovlen = 1
+	if len(iov) > 0 {
+		msg.Iov = &iov[0]
+		msg.SetIovlen(len(iov))
+	}
 	if n, err = recvmsg(fd, &msg, flags); n == -1 {
 		return
 	}
@@ -480,30 +479,31 @@ func recvmsgRaw(fd int, p, oob []byte, flags int, rsa *RawSockaddrAny) (n, oobn
 
 //sys	sendmsg(s int, msg *Msghdr, flags int) (n int, err error) = libsocket.__xnet_sendmsg
 
-func sendmsgN(fd int, p, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
+func sendmsgN(fd int, iov []Iovec, oob []byte, ptr unsafe.Pointer, salen _Socklen, flags int) (n int, err error) {
 	var msg Msghdr
 	msg.Name = (*byte)(unsafe.Pointer(ptr))
 	msg.Namelen = uint32(salen)
-	var iov Iovec
-	if len(p) > 0 {
-		iov.Base = (*int8)(unsafe.Pointer(&p[0]))
-		iov.SetLen(len(p))
-	}
-	var dummy int8
+	var dummy byte
+	var empty bool
 	if len(oob) > 0 {
 		// send at least one normal byte
-		if len(p) == 0 {
-			iov.Base = &dummy
-			iov.SetLen(1)
+		empty = emptyIovecs(iov)
+		if empty {
+			var iova [1]Iovec
+			iova[0].Base = &dummy
+			iova[0].SetLen(1)
+			iov = iova[:]
 		}
 		msg.Accrightslen = int32(len(oob))
 	}
-	msg.Iov = &iov
-	msg.Iovlen = 1
+	if len(iov) > 0 {
+		msg.Iov = &iov[0]
+		msg.SetIovlen(len(iov))
+	}
 	if n, err = sendmsg(fd, &msg, flags); err != nil {
 		return 0, err
 	}
-	if len(oob) > 0 && len(p) == 0 {
+	if len(oob) > 0 && empty {
 		n = 0
 	}
 	return n, nil
diff --git a/vendor/golang.org/x/sys/unix/syscall_unix.go b/vendor/golang.org/x/sys/unix/syscall_unix.go
index 70508afc1d..1ff5060b51 100644
--- a/vendor/golang.org/x/sys/unix/syscall_unix.go
+++ b/vendor/golang.org/x/sys/unix/syscall_unix.go
@@ -338,8 +338,13 @@ func Recvfrom(fd int, p []byte, flags int) (n int, from Sockaddr, err error) {
 }
 
 func Recvmsg(fd int, p, oob []byte, flags int) (n, oobn int, recvflags int, from Sockaddr, err error) {
+	var iov [1]Iovec
+	if len(p) > 0 {
+		iov[0].Base = &p[0]
+		iov[0].SetLen(len(p))
+	}
 	var rsa RawSockaddrAny
-	n, oobn, recvflags, err = recvmsgRaw(fd, p, oob, flags, &rsa)
+	n, oobn, recvflags, err = recvmsgRaw(fd, iov[:], oob, flags, &rsa)
 	// source address is only specified if the socket is unconnected
 	if rsa.Addr.Family != AF_UNSPEC {
 		from, err = anyToSockaddr(fd, &rsa)
@@ -347,12 +352,42 @@ func Recvmsg(fd int, p, oob []byte, flags int) (n, oobn int, recvflags int, from
 	return
 }
 
+// RecvmsgBuffers receives a message from a socket using the recvmsg
+// system call. The flags are passed to recvmsg. Any non-control data
+// read is scattered into the buffers slices. The results are:
+//   - n is the number of non-control data read into bufs
+//   - oobn is the number of control data read into oob; this may be interpreted using [ParseSocketControlMessage]
+//   - recvflags is flags returned by recvmsg
+//   - from is the address of the sender
+func RecvmsgBuffers(fd int, buffers [][]byte, oob []byte, flags int) (n, oobn int, recvflags int, from Sockaddr, err error) {
+	iov := make([]Iovec, len(buffers))
+	for i := range buffers {
+		if len(buffers[i]) > 0 {
+			iov[i].Base = &buffers[i][0]
+			iov[i].SetLen(len(buffers[i]))
+		} else {
+			iov[i].Base = (*byte)(unsafe.Pointer(&_zero))
+		}
+	}
+	var rsa RawSockaddrAny
+	n, oobn, recvflags, err = recvmsgRaw(fd, iov, oob, flags, &rsa)
+	if err == nil && rsa.Addr.Family != AF_UNSPEC {
+		from, err = anyToSockaddr(fd, &rsa)
+	}
+	return
+}
+
 func Sendmsg(fd int, p, oob []byte, to Sockaddr, flags int) (err error) {
 	_, err = SendmsgN(fd, p, oob, to, flags)
 	return
 }
 
 func SendmsgN(fd int, p, oob []byte, to Sockaddr, flags int) (n int, err error) {
+	var iov [1]Iovec
+	if len(p) > 0 {
+		iov[0].Base = &p[0]
+		iov[0].SetLen(len(p))
+	}
 	var ptr unsafe.Pointer
 	var salen _Socklen
 	if to != nil {
@@ -361,7 +396,32 @@ func SendmsgN(fd int, p, oob []byte, to Sockaddr, flags int) (n int, err error)
 			return 0, err
 		}
 	}
-	return sendmsgN(fd, p, oob, ptr, salen, flags)
+	return sendmsgN(fd, iov[:], oob, ptr, salen, flags)
+}
+
+// SendmsgBuffers sends a message on a socket to an address using the sendmsg
+// system call. The flags are passed to sendmsg. Any non-control data written
+// is gathered from buffers. The function returns the number of bytes written
+// to the socket.
+func SendmsgBuffers(fd int, buffers [][]byte, oob []byte, to Sockaddr, flags int) (n int, err error) {
+	iov := make([]Iovec, len(buffers))
+	for i := range buffers {
+		if len(buffers[i]) > 0 {
+			iov[i].Base = &buffers[i][0]
+			iov[i].SetLen(len(buffers[i]))
+		} else {
+			iov[i].Base = (*byte)(unsafe.Pointer(&_zero))
+		}
+	}
+	var ptr unsafe.Pointer
+	var salen _Socklen
+	if to != nil {
+		ptr, salen, err = to.sockaddr()
+		if err != nil {
+			return 0, err
+		}
+	}
+	return sendmsgN(fd, iov, oob, ptr, salen, flags)
 }
 
 func Send(s int, buf []byte, flags int) (err error) {
@@ -484,3 +544,13 @@ func Lutimes(path string, tv []Timeval) error {
 	}
 	return UtimesNanoAt(AT_FDCWD, path, ts, AT_SYMLINK_NOFOLLOW)
 }
+
+// emptyIovec reports whether there are no bytes in the slice of Iovec.
+func emptyIovecs(iov []Iovec) bool {
+	for i := range iov {
+		if iov[i].Len > 0 {
+			return false
+		}
+	}
+	return true
+}
diff --git a/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go
index ad4aad2796..c1a9b83ad5 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go
@@ -178,7 +178,7 @@ type Linger struct {
 }
 
 type Iovec struct {
-	Base *int8
+	Base *byte
 	Len  uint64
 }
 
diff --git a/vendor/google.golang.org/api/internal/gensupport/resumable.go b/vendor/google.golang.org/api/internal/gensupport/resumable.go
index 0eae147fa9..0c659188dd 100644
--- a/vendor/google.golang.org/api/internal/gensupport/resumable.go
+++ b/vendor/google.golang.org/api/internal/gensupport/resumable.go
@@ -10,8 +10,12 @@ import (
 	"fmt"
 	"io"
 	"net/http"
+	"strings"
 	"sync"
 	"time"
+
+	"github.com/google/uuid"
+	"google.golang.org/api/internal"
 )
 
 // ResumableUpload is used by the generated APIs to provide resumable uploads.
@@ -38,6 +42,11 @@ type ResumableUpload struct {
 	// ChunkRetryDeadline configures the per-chunk deadline after which no further
 	// retries should happen.
 	ChunkRetryDeadline time.Duration
+
+	// Track current request invocation ID and attempt count for retry metric
+	// headers.
+	invocationID string
+	attempts     int
 }
 
 // Progress returns the number of bytes uploaded at this point.
@@ -72,6 +81,10 @@ func (rx *ResumableUpload) doUploadRequest(ctx context.Context, data io.Reader,
 	req.Header.Set("Content-Type", rx.MediaType)
 	req.Header.Set("User-Agent", rx.UserAgent)
 
+	baseXGoogHeader := "gl-go/" + GoVersion() + " gdcl/" + internal.Version
+	invocationHeader := fmt.Sprintf("gccl-invocation-id/%s gccl-attempt-count/%d", rx.invocationID, rx.attempts)
+	req.Header.Set("X-Goog-Api-Client", strings.Join([]string{baseXGoogHeader, invocationHeader}, " "))
+
 	// Google's upload endpoint uses status code 308 for a
 	// different purpose than the "308 Permanent Redirect"
 	// since-standardized in RFC 7238. Because of the conflict in
@@ -178,9 +191,11 @@ func (rx *ResumableUpload) Upload(ctx context.Context) (resp *http.Response, err
 	for {
 		var pause time.Duration
 
-		// Each chunk gets its own initialized-at-zero backoff.
+		// Each chunk gets its own initialized-at-zero backoff and invocation ID.
 		bo := rx.Retry.backoff()
 		quitAfter := time.After(retryDeadline)
+		rx.attempts = 1
+		rx.invocationID = uuid.New().String()
 
 		// Retry loop for a single chunk.
 		for {
@@ -223,6 +238,7 @@ func (rx *ResumableUpload) Upload(ctx context.Context) (resp *http.Response, err
 				break
 			}
 
+			rx.attempts++
 			pause = bo.Pause()
 			if resp != nil && resp.Body != nil {
 				resp.Body.Close()
diff --git a/vendor/google.golang.org/api/internal/gensupport/send.go b/vendor/google.golang.org/api/internal/gensupport/send.go
index dd50cc20a5..70a8e01c1b 100644
--- a/vendor/google.golang.org/api/internal/gensupport/send.go
+++ b/vendor/google.golang.org/api/internal/gensupport/send.go
@@ -8,9 +8,12 @@ import (
 	"context"
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net/http"
+	"strings"
 	"time"
 
+	"github.com/google/uuid"
 	"github.com/googleapis/gax-go/v2"
 )
 
@@ -71,6 +74,9 @@ func sendAndRetry(ctx context.Context, client *http.Client, req *http.Request, r
 
 	var resp *http.Response
 	var err error
+	attempts := 1
+	invocationID := uuid.New().String()
+	baseXGoogHeader := req.Header.Get("X-Goog-Api-Client")
 
 	// Loop to retry the request, up to the context deadline.
 	var pause time.Duration
@@ -109,6 +115,9 @@ func sendAndRetry(ctx context.Context, client *http.Client, req *http.Request, r
 			}
 			return resp, err
 		}
+		invocationHeader := fmt.Sprintf("gccl-invocation-id/%s gccl-attempt-count/%d", invocationID, attempts)
+		xGoogHeader := strings.Join([]string{invocationHeader, baseXGoogHeader}, " ")
+		req.Header.Set("X-Goog-Api-Client", xGoogHeader)
 
 		resp, err = client.Do(req.WithContext(ctx))
 
@@ -123,6 +132,7 @@ func sendAndRetry(ctx context.Context, client *http.Client, req *http.Request, r
 		if req.GetBody == nil || !errorFunc(status, err) {
 			break
 		}
+		attempts++
 		var errBody error
 		req.Body, errBody = req.GetBody()
 		if errBody != nil {
diff --git a/vendor/google.golang.org/api/internal/version.go b/vendor/google.golang.org/api/internal/version.go
index be0f776af7..40b776bd77 100644
--- a/vendor/google.golang.org/api/internal/version.go
+++ b/vendor/google.golang.org/api/internal/version.go
@@ -5,4 +5,4 @@
 package internal
 
 // Version is the current tagged release of the library.
-const Version = "0.84.0"
+const Version = "0.85.0"
diff --git a/vendor/modules.txt b/vendor/modules.txt
index f76d20b22c..a8814f5c6f 100644
--- a/vendor/modules.txt
+++ b/vendor/modules.txt
@@ -11,7 +11,7 @@ cloud.google.com/go/compute/metadata
 # cloud.google.com/go/iam v0.3.0
 ## explicit; go 1.15
 cloud.google.com/go/iam
-# cloud.google.com/go/storage v1.22.1
+# cloud.google.com/go/storage v1.23.0
 ## explicit; go 1.15
 cloud.google.com/go/storage
 cloud.google.com/go/storage/internal
@@ -34,7 +34,7 @@ github.com/VictoriaMetrics/metricsql/binaryop
 # github.com/VividCortex/ewma v1.2.0
 ## explicit; go 1.12
 github.com/VividCortex/ewma
-# github.com/aws/aws-sdk-go v1.44.37
+# github.com/aws/aws-sdk-go v1.44.43
 ## explicit; go 1.11
 github.com/aws/aws-sdk-go/aws
 github.com/aws/aws-sdk-go/aws/arn
@@ -229,7 +229,7 @@ github.com/rivo/uniseg
 # github.com/russross/blackfriday/v2 v2.1.0
 ## explicit
 github.com/russross/blackfriday/v2
-# github.com/urfave/cli/v2 v2.10.1
+# github.com/urfave/cli/v2 v2.10.3
 ## explicit; go 1.18
 github.com/urfave/cli/v2
 # github.com/valyala/bytebufferpool v1.0.0
@@ -281,7 +281,7 @@ go.opencensus.io/trace/tracestate
 go.uber.org/atomic
 # go.uber.org/goleak v1.1.11-0.20210813005559-691160354723
 ## explicit; go 1.13
-# golang.org/x/net v0.0.0-20220617184016-355a448f1bc9
+# golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e
 ## explicit; go 1.17
 golang.org/x/net/context
 golang.org/x/net/context/ctxhttp
@@ -293,7 +293,7 @@ golang.org/x/net/internal/socks
 golang.org/x/net/internal/timeseries
 golang.org/x/net/proxy
 golang.org/x/net/trace
-# golang.org/x/oauth2 v0.0.0-20220608161450-d0670ef3b1eb
+# golang.org/x/oauth2 v0.0.0-20220622183110-fd043fe589d2
 ## explicit; go 1.15
 golang.org/x/oauth2
 golang.org/x/oauth2/authhandler
@@ -306,7 +306,7 @@ golang.org/x/oauth2/jwt
 # golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f
 ## explicit
 golang.org/x/sync/errgroup
-# golang.org/x/sys v0.0.0-20220615213510-4f61da869c0c
+# golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b
 ## explicit; go 1.17
 golang.org/x/sys/internal/unsafeheader
 golang.org/x/sys/unix
@@ -321,7 +321,7 @@ golang.org/x/text/unicode/norm
 ## explicit; go 1.17
 golang.org/x/xerrors
 golang.org/x/xerrors/internal
-# google.golang.org/api v0.84.0
+# google.golang.org/api v0.85.0
 ## explicit; go 1.15
 google.golang.org/api/googleapi
 google.golang.org/api/googleapi/transport
@@ -354,7 +354,7 @@ google.golang.org/appengine/internal/socket
 google.golang.org/appengine/internal/urlfetch
 google.golang.org/appengine/socket
 google.golang.org/appengine/urlfetch
-# google.golang.org/genproto v0.0.0-20220617124728-180714bec0ad
+# google.golang.org/genproto v0.0.0-20220627200112-0a929928cb33
 ## explicit; go 1.15
 google.golang.org/genproto/googleapis/api/annotations
 google.golang.org/genproto/googleapis/iam/v1