Merge branch 'public-single-node' into pmm-6401-read-prometheus-data-files

This commit is contained in:
Aliaksandr Valialkin 2022-06-28 20:22:11 +03:00
commit e7b4e657a1
No known key found for this signature in database
GPG key ID: A72BEC6CD3D0DED1
45 changed files with 2307 additions and 940 deletions

View file

@ -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 {

View file

@ -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.

View file

@ -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
}

View file

@ -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) %}

View file

@ -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
}

View file

@ -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

View file

@ -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 {

View file

@ -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)

View file

@ -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.

View file

@ -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.

View file

@ -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).
details [here](https://docs.victoriametrics.com/Single-server-VictoriaMetrics.html#deduplication).

16
go.mod
View file

@ -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
)

26
go.sum
View file

@ -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=

View file

@ -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.

View file

@ -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.

View file

@ -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")

View file

@ -1,3 +1,3 @@
{
"storage": "1.22.1"
"storage": "1.23.0"
}

View file

@ -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)

View file

@ -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

View file

@ -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(),
}
}

View file

@ -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

View file

@ -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
}

View file

@ -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()
}

View file

@ -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 objects
// metadata.
func (c *Client) WriteObject(ctx context.Context, opts ...gax.CallOption) (storagepb.Storage_WriteObjectClient, error) {
return c.internalClient.WriteObject(ctx, opts...)
}

View file

@ -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"

View file

@ -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")

View file

@ -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.

View file

@ -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)

View file

@ -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
}

View file

@ -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{

View file

@ -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"

View file

@ -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...)
}

View file

@ -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
}

View file

@ -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
}

View file

@ -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

View file

@ -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

View file

@ -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

View file

@ -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)
}

View file

@ -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

View file

@ -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
}

View file

@ -178,7 +178,7 @@ type Linger struct {
}
type Iovec struct {
Base *int8
Base *byte
Len uint64
}

View file

@ -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()

View file

@ -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 {

View file

@ -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"

16
vendor/modules.txt vendored
View file

@ -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