Skip to content

Commit

Permalink
Merge pull request #63540 from knz/backport21.1-63327
Browse files Browse the repository at this point in the history
  • Loading branch information
knz authored May 12, 2021
2 parents 17de9a2 + 683fc04 commit 34f4ba6
Show file tree
Hide file tree
Showing 4 changed files with 115 additions and 36 deletions.
2 changes: 2 additions & 0 deletions pkg/util/timeutil/pgdate/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
],
)

Expand All @@ -39,6 +40,7 @@ go_test(
embed = [":pgdate"],
deps = [
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_lib_pq//:pq",
],
)
Expand Down
77 changes: 46 additions & 31 deletions pkg/util/timeutil/pgdate/field_extract.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,14 +11,14 @@
package pgdate

import (
"fmt"
"strconv"
"strings"
"time"
"unicode"
"unicode/utf8"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// numberChunk associates a value with a leading separator,
Expand All @@ -31,11 +31,15 @@ type numberChunk struct {
magnitude int
}

func (n numberChunk) String() string {
func (n numberChunk) String() string { return redact.StringWithoutMarkers(n) }

// SafeFormat implements the redact.SafeFormatter interface.
func (n numberChunk) SafeFormat(w redact.SafePrinter, _ rune) {
if n.separator == utf8.RuneError {
return fmt.Sprintf("%d", n.v)
w.Print(n.v)
return
}
return fmt.Sprintf("%v%d", n.separator, n.v)
w.Printf("%c%d", n.separator, n.v)
}

// fieldExtract manages the state of a date/time parsing operation.
Expand Down Expand Up @@ -336,7 +340,7 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
return fe.SetChunk(fieldNanos, chunk)

default:
return inputErrorf("cannot interpret field: %s", chunk)
return fe.decorateError(inputErrorf("cannot interpret field: %s", chunk))
}

case chunk.magnitude == 3 &&
Expand Down Expand Up @@ -534,7 +538,7 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
return fe.SetChunk(fieldTZHour, chunk)

default:
return inputErrorf("unexpected number of digits for timezone in: %s", chunk)
return fe.decorateError(inputErrorf("unexpected number of digits for timezone in: %s", chunk))
}

case !fe.Wants(fieldTZHour) && fe.Wants(fieldTZMinute):
Expand Down Expand Up @@ -609,7 +613,7 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
return fe.SetChunk(fieldHour, chunk)

default:
return inputErrorf("unexpected number of digits for time in %v", chunk)
return fe.decorateError(inputErrorf("unexpected number of digits for time in %v", chunk))
}

case fe.Wants(fieldMinute):
Expand All @@ -626,7 +630,7 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
return fe.SetChunk(fieldSecond, chunk)
}
}
return inputErrorf("could not parse field: %v", chunk)
return fe.decorateError(inputErrorf("could not parse field: %v", chunk))
}

// MakeDate returns a time.Time containing only the date components
Expand Down Expand Up @@ -755,7 +759,7 @@ func (fe *fieldExtract) matchedSentinel(value time.Time, match string) error {
// Reset replaces a value of an already-set field.
func (fe *fieldExtract) Reset(field field, v int) error {
if !fe.has.Has(field) {
return errors.AssertionFailedf("field %s is not already set", errors.Safe(field.Pretty()))
return errors.AssertionFailedf("field %s is not already set", field.SafePretty())
}
fe.data[field] = v
return nil
Expand All @@ -765,7 +769,9 @@ func (fe *fieldExtract) Reset(field field, v int) error {
// the field has already been set.
func (fe *fieldExtract) Set(field field, v int) error {
if !fe.wanted.Has(field) {
return errors.AssertionFailedf("field %s is not wanted in %v", errors.Safe(field.Pretty()), errors.Safe(fe.wanted))
return fe.decorateError(
inputErrorf("value %v for field %s already present or not wanted", v, field.SafePretty()),
)
}
fe.data[field] = v
fe.has = fe.has.Add(field)
Expand All @@ -774,6 +780,12 @@ func (fe *fieldExtract) Set(field field, v int) error {
return nil
}

// decorateError adds context to an error object.
func (fe *fieldExtract) decorateError(err error) error {
return errors.WithDetailf(err,
"Wanted: %v\nAlready found in input: %v", &fe.wanted, &fe.has)
}

// SetChunk first validates that the separator in the chunk is appropriate
// for the field being set. That is, you'd never see "YYYY:MM:DD";
// colons are only appropriate for hours and minutes.
Expand Down Expand Up @@ -832,14 +844,14 @@ func (fe *fieldExtract) SetChunk(field field, chunk numberChunk) error {
return fe.Set(field, chunk.v)
}
}
return badFieldPrefixError(field, chunk.separator)
return fe.decorateError(badFieldPrefixError(field, chunk.separator))
}

// SetDayOfYear updates the month and day fields to reflect the
// given day-of-year. The year must have been previously set.
func (fe *fieldExtract) SetDayOfYear(chunk numberChunk) error {
if chunk.separator != ' ' && chunk.separator != '.' {
return badFieldPrefixError(fieldMonth, chunk.separator)
return fe.decorateError(badFieldPrefixError(fieldMonth, chunk.separator))
}

y, ok := fe.Get(fieldYear)
Expand All @@ -856,31 +868,33 @@ func (fe *fieldExtract) SetDayOfYear(chunk numberChunk) error {
return fe.Set(fieldDay, d)
}

func (fe *fieldExtract) String() string {
ret := "[ "
// SafeFormat implements the redact.SafeFormatter interface.
func (fe *fieldExtract) SafeFormat(w redact.SafePrinter, _ rune) {
w.SafeString("[ ")
for f := fieldMinimum; f <= fieldMaximum; f++ {
if v, ok := fe.Get(f); ok {
ret += fmt.Sprintf("%s: %d ", f.Pretty(), v)
w.Printf("%s: %d ", f.SafePretty(), v)
}
}
ret += "]"
return ret
w.SafeRune(']')
}

func (fe *fieldExtract) String() string { return redact.StringWithoutMarkers(fe) }

// validate ensures that the data in the extract is reasonable. It also
// performs some field fixups, such as converting two-digit years
// to actual values and adjusting for AM/PM.
func (fe *fieldExtract) validate() error {
// If we have any of the required fields, we must have all of the required fields.
if fe.has.HasAny(dateRequiredFields) && !fe.has.HasAll(dateRequiredFields) {
return inputErrorf("missing required date fields")
return fe.decorateError(inputErrorf("missing required date fields"))
}

if (fe.isDB2 && !fe.has.HasAll(db2TimeRequiredFields)) || (fe.has.HasAny(timeRequiredFields) && !fe.has.HasAll(timeRequiredFields)) {
return inputErrorf("missing required time fields")
return fe.decorateError(inputErrorf("missing required time fields"))
}
if !fe.has.HasAll(fe.required) {
return inputErrorf("missing required fields in input")
return fe.decorateError(inputErrorf("missing required fields in input"))
}

if year, ok := fe.Get(fieldYear); ok {
Expand All @@ -891,7 +905,8 @@ func (fe *fieldExtract) validate() error {

if era, ok := fe.Get(fieldEra); ok {
if year <= 0 {
return inputErrorf("only positive years are permitted in AD/BC notation")
return fe.decorateError(
inputErrorf("only positive years are permitted in AD/BC notation (%v)", year))
}
if era < 0 {
// Update for BC dates.
Expand All @@ -901,7 +916,7 @@ func (fe *fieldExtract) validate() error {
}
} else if fe.tweakYear {
if year < 0 {
return inputErrorf("negative year not allowed")
return inputErrorf("negative year (%v) not allowed", year)
}
if year < 70 {
year += 2000
Expand All @@ -915,7 +930,7 @@ func (fe *fieldExtract) validate() error {

if month, ok := fe.Get(fieldMonth); ok {
if month < 1 || month > 12 {
return outOfRangeError("month", month)
return fe.decorateError(outOfRangeError("month", month))
}

if day, ok := fe.Get(fieldDay); ok {
Expand All @@ -926,7 +941,7 @@ func (fe *fieldExtract) validate() error {
maxDay = daysInMonth[0][month]
}
if day < 1 || day > maxDay {
return outOfRangeError("day", day)
return fe.decorateError(outOfRangeError("day", day))
}
}
}
Expand All @@ -940,7 +955,7 @@ func (fe *fieldExtract) validate() error {
case fieldValueAM:
switch {
case hour < 0 || hour > 12:
return outOfRangeError("hour", hour)
return fe.decorateError(outOfRangeError("hour", hour))
case hour == 12:
if err := fe.Reset(fieldHour, 0); err != nil {
return err
Expand All @@ -950,7 +965,7 @@ func (fe *fieldExtract) validate() error {
case fieldValuePM:
switch {
case hour < 0 || hour > 12:
return outOfRangeError("hour", hour)
return fe.decorateError(outOfRangeError("hour", hour))
case hour == 12:
// 12 PM -> 12
default:
Expand All @@ -963,31 +978,31 @@ func (fe *fieldExtract) validate() error {
default:
// 24:00:00 is the maximum-allowed value
if hour < 0 || (hasDate && hour > 24) || (!hasDate && hour > 23) {
return outOfRangeError("hour", hour)
return fe.decorateError(outOfRangeError("hour", hour))
}
}

minute, _ := fe.Get(fieldMinute)
if minute < 0 || minute > 59 {
return outOfRangeError("minute", minute)
return fe.decorateError(outOfRangeError("minute", minute))
}

second, _ := fe.Get(fieldSecond)
if second < 0 || (hasDate && second > 60) || (!hasDate && second > 59) {
return outOfRangeError("second", second)
return fe.decorateError(outOfRangeError("second", second))
}

nanos, _ := fe.Get(fieldNanos)
if nanos < 0 {
return outOfRangeError("nanos", nanos)
return fe.decorateError(outOfRangeError("nanos", nanos))
}

x := time.Duration(hour)*time.Hour +
time.Duration(minute)*time.Minute +
time.Duration(second)*time.Second +
time.Duration(nanos)*time.Nanosecond
if x > 24*time.Hour {
return inputErrorf("time out of range: %d", x)
return fe.decorateError(inputErrorf("time out of range: %d", x))
}
}

Expand Down
50 changes: 50 additions & 0 deletions pkg/util/timeutil/pgdate/field_extract_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,12 @@ package pgdate

import (
"reflect"
"strings"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

func TestExtractRelative(t *testing.T) {
Expand Down Expand Up @@ -86,6 +88,10 @@ func TestExtractSentinels(t *testing.T) {
s: keywordNow + " tomorrow",
err: true,
},
{
s: "j66001",
err: true,
},
}
for _, tc := range tests {
t.Run(tc.s, func(t *testing.T) {
Expand All @@ -107,6 +113,50 @@ func TestExtractSentinels(t *testing.T) {
}
}

func TestExtractInvalidJulianDate(t *testing.T) {
testCases := []struct {
wanted fieldSet
str string
expError string
}{
// Expect only a time, reject a year.
{timeFields, "j69001", `value -4524 for field Year already present or not wanted
Wanted: [ Hour Minute Second Nanos Meridian TZHour TZMinute TZSecond ]
Already found in input: [ ]`},
// Expect a date, reject when the year is specified twice.
{dateFields, "j69001 j69002", `value -4524 for field Year already present or not wanted
Wanted: [ Era ]
Already found in input: [ Year Month Day ]`},
/// Expect a date+time, reject when the date/month/day is specified twice.
{dateTimeFields, "2010-10-10 j69002", `could not parse field: -10
Wanted: [ Era Nanos Meridian ]
Already found in input: [ Year Month Day Hour Minute Second TZHour TZMinute TZSecond ]`},
}

// TODO(knz): This would benefit from datadriven testing.

now := timeutil.Unix(42, 56)
for _, tc := range testCases {
fe := fieldExtract{
currentTime: now,
wanted: tc.wanted,
}
err := fe.Extract(tc.str)
if err == nil {
t.Errorf("%+v: expected error, got no error", tc)
continue
}

msg := err.Error()
msg += "\n" + errors.FlattenDetails(err)
if msg != tc.expError {
t.Errorf("expected error:\n %v\ngot:\n %v",
strings.ReplaceAll(tc.expError, "\n", "\n "),
strings.ReplaceAll(msg, "\n", "\n "))
}
}
}

func TestFieldExtractSet(t *testing.T) {
p := fieldExtract{wanted: dateFields}
if err := p.Set(fieldYear, 2018); err != nil {
Expand Down
22 changes: 17 additions & 5 deletions pkg/util/timeutil/pgdate/fields.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@

package pgdate

import "github.com/cockroachdb/redact"

//go:generate stringer -type=field

// A field is some piece of information that we can newFieldExtract out of a
Expand Down Expand Up @@ -54,6 +56,14 @@ func (f field) Pretty() string {
return f.String()[5:]
}

// SafePretty wraps the generated String() function to return only the
// name: "Year" vs "fieldYear" as a RedactableString.
func (f field) SafePretty() redact.RedactableString {
// Note: this cast is safe because the String() method is generated
// by the stringer and only returns literal strings.
return redact.RedactableString(f.String()[5:])
}

// A fieldSet is an immutable aggregate of fields.
// The zero value is an empty set.
type fieldSet int
Expand Down Expand Up @@ -106,13 +116,15 @@ func (s fieldSet) HasAny(other fieldSet) bool {
return s&other != 0
}

func (s *fieldSet) String() string {
ret := "[ "
func (s *fieldSet) String() string { return redact.StringWithoutMarkers(s) }

// SafeFormat implements the redact.SafeFormatter interface.
func (s *fieldSet) SafeFormat(w redact.SafePrinter, _ rune) {
w.SafeString("[ ")
for f := fieldMinimum; f <= fieldMaximum; f++ {
if s.Has(f) {
ret += f.Pretty() + " "
w.Printf("%v ", f.SafePretty())
}
}
ret += "]"
return ret
w.SafeRune(']')
}

0 comments on commit 34f4ba6

Please sign in to comment.