mirror of
https://github.com/minio/minio.git
synced 2025-11-07 21:02:58 -05:00
rename all remaining packages to internal/ (#12418)
This is to ensure that there are no projects that try to import `minio/minio/pkg` into their own repo. Any such common packages should go to `https://github.com/minio/pkg`
This commit is contained in:
96
internal/s3select/json/args.go
Normal file
96
internal/s3select/json/args.go
Normal file
@@ -0,0 +1,96 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package json
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"fmt"
|
||||
"strings"
|
||||
)
|
||||
|
||||
const (
|
||||
document = "document"
|
||||
lines = "lines"
|
||||
|
||||
defaultRecordDelimiter = "\n"
|
||||
)
|
||||
|
||||
// ReaderArgs - represents elements inside <InputSerialization><JSON/> in request XML.
|
||||
type ReaderArgs struct {
|
||||
ContentType string `xml:"Type"`
|
||||
unmarshaled bool
|
||||
}
|
||||
|
||||
// IsEmpty - returns whether reader args is empty or not.
|
||||
func (args *ReaderArgs) IsEmpty() bool {
|
||||
return !args.unmarshaled
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (args *ReaderArgs) UnmarshalXML(d *xml.Decoder, start xml.StartElement) error {
|
||||
// Make subtype to avoid recursive UnmarshalXML().
|
||||
type subReaderArgs ReaderArgs
|
||||
parsedArgs := subReaderArgs{}
|
||||
if err := d.DecodeElement(&parsedArgs, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
parsedArgs.ContentType = strings.ToLower(parsedArgs.ContentType)
|
||||
switch parsedArgs.ContentType {
|
||||
case document, lines:
|
||||
default:
|
||||
return errInvalidJSONType(fmt.Errorf("invalid ContentType '%v'", parsedArgs.ContentType))
|
||||
}
|
||||
|
||||
*args = ReaderArgs(parsedArgs)
|
||||
args.unmarshaled = true
|
||||
return nil
|
||||
}
|
||||
|
||||
// WriterArgs - represents elements inside <OutputSerialization><JSON/> in request XML.
|
||||
type WriterArgs struct {
|
||||
RecordDelimiter string `xml:"RecordDelimiter"`
|
||||
unmarshaled bool
|
||||
}
|
||||
|
||||
// IsEmpty - returns whether writer args is empty or not.
|
||||
func (args *WriterArgs) IsEmpty() bool {
|
||||
return !args.unmarshaled
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (args *WriterArgs) UnmarshalXML(d *xml.Decoder, start xml.StartElement) error {
|
||||
// Make subtype to avoid recursive UnmarshalXML().
|
||||
type subWriterArgs WriterArgs
|
||||
parsedArgs := subWriterArgs{}
|
||||
if err := d.DecodeElement(&parsedArgs, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
switch len(parsedArgs.RecordDelimiter) {
|
||||
case 0:
|
||||
parsedArgs.RecordDelimiter = defaultRecordDelimiter
|
||||
case 1, 2:
|
||||
default:
|
||||
return fmt.Errorf("invalid RecordDelimiter '%v'", parsedArgs.RecordDelimiter)
|
||||
}
|
||||
|
||||
*args = WriterArgs(parsedArgs)
|
||||
args.unmarshaled = true
|
||||
return nil
|
||||
}
|
||||
63
internal/s3select/json/errors.go
Normal file
63
internal/s3select/json/errors.go
Normal file
@@ -0,0 +1,63 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package json
|
||||
|
||||
type s3Error struct {
|
||||
code string
|
||||
message string
|
||||
statusCode int
|
||||
cause error
|
||||
}
|
||||
|
||||
func (err *s3Error) Cause() error {
|
||||
return err.cause
|
||||
}
|
||||
|
||||
func (err *s3Error) ErrorCode() string {
|
||||
return err.code
|
||||
}
|
||||
|
||||
func (err *s3Error) ErrorMessage() string {
|
||||
return err.message
|
||||
}
|
||||
|
||||
func (err *s3Error) HTTPStatusCode() int {
|
||||
return err.statusCode
|
||||
}
|
||||
|
||||
func (err *s3Error) Error() string {
|
||||
return err.message
|
||||
}
|
||||
|
||||
func errInvalidJSONType(err error) *s3Error {
|
||||
return &s3Error{
|
||||
code: "InvalidJsonType",
|
||||
message: "The JsonType is invalid. Only DOCUMENT and LINES are supported.",
|
||||
statusCode: 400,
|
||||
cause: err,
|
||||
}
|
||||
}
|
||||
|
||||
func errJSONParsingError(err error) *s3Error {
|
||||
return &s3Error{
|
||||
code: "JSONParsingError",
|
||||
message: "Encountered an error parsing the JSON file. Check the file and try again.",
|
||||
statusCode: 400,
|
||||
cause: err,
|
||||
}
|
||||
}
|
||||
228
internal/s3select/json/preader.go
Normal file
228
internal/s3select/json/preader.go
Normal file
@@ -0,0 +1,228 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package json
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"bytes"
|
||||
"io"
|
||||
"runtime"
|
||||
"sync"
|
||||
|
||||
"github.com/bcicen/jstream"
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
)
|
||||
|
||||
// PReader - JSON record reader for S3Select.
|
||||
// Operates concurrently on line-delimited JSON.
|
||||
type PReader struct {
|
||||
args *ReaderArgs
|
||||
readCloser io.ReadCloser // raw input
|
||||
buf *bufio.Reader // input to the splitter
|
||||
current []jstream.KVS // current block of results to be returned
|
||||
recordsRead int // number of records read in current slice
|
||||
input chan *queueItem // input for workers
|
||||
queue chan *queueItem // output from workers in order
|
||||
err error // global error state, only touched by Reader.Read
|
||||
bufferPool sync.Pool // pool of []byte objects for input
|
||||
kvDstPool sync.Pool // pool of []jstream.KV used for output
|
||||
close chan struct{} // used for shutting down the splitter before end of stream
|
||||
readerWg sync.WaitGroup // used to keep track of async reader.
|
||||
}
|
||||
|
||||
// queueItem is an item in the queue.
|
||||
type queueItem struct {
|
||||
input []byte // raw input sent to the worker
|
||||
dst chan []jstream.KVS // result of block decode
|
||||
err error // any error encountered will be set here
|
||||
}
|
||||
|
||||
// Read - reads single record.
|
||||
// Once Read is called the previous record should no longer be referenced.
|
||||
func (r *PReader) Read(dst sql.Record) (sql.Record, error) {
|
||||
// If we have have any records left, return these before any error.
|
||||
for len(r.current) <= r.recordsRead {
|
||||
if r.err != nil {
|
||||
return nil, r.err
|
||||
}
|
||||
// Move to next block
|
||||
item, ok := <-r.queue
|
||||
if !ok {
|
||||
r.err = io.EOF
|
||||
return nil, r.err
|
||||
}
|
||||
//lint:ignore SA6002 Using pointer would allocate more since we would have to copy slice header before taking a pointer.
|
||||
r.kvDstPool.Put(r.current)
|
||||
r.current = <-item.dst
|
||||
r.err = item.err
|
||||
r.recordsRead = 0
|
||||
}
|
||||
kvRecord := r.current[r.recordsRead]
|
||||
r.recordsRead++
|
||||
|
||||
dstRec, ok := dst.(*Record)
|
||||
if !ok {
|
||||
dstRec = &Record{}
|
||||
}
|
||||
dstRec.KVS = kvRecord
|
||||
dstRec.SelectFormat = sql.SelectFmtJSON
|
||||
return dstRec, nil
|
||||
}
|
||||
|
||||
// Close - closes underlying reader.
|
||||
func (r *PReader) Close() error {
|
||||
if r.close != nil {
|
||||
close(r.close)
|
||||
r.readerWg.Wait()
|
||||
r.close = nil
|
||||
}
|
||||
r.recordsRead = len(r.current)
|
||||
if r.err == nil {
|
||||
r.err = io.EOF
|
||||
}
|
||||
return r.readCloser.Close()
|
||||
}
|
||||
|
||||
// nextSplit will attempt to skip a number of bytes and
|
||||
// return the buffer until the next newline occurs.
|
||||
// The last block will be sent along with an io.EOF.
|
||||
func (r *PReader) nextSplit(skip int, dst []byte) ([]byte, error) {
|
||||
if cap(dst) < skip {
|
||||
dst = make([]byte, 0, skip+1024)
|
||||
}
|
||||
dst = dst[:skip]
|
||||
if skip > 0 {
|
||||
n, err := io.ReadFull(r.buf, dst)
|
||||
if err != nil && err != io.ErrUnexpectedEOF {
|
||||
// If an EOF happens after reading some but not all the bytes,
|
||||
// ReadFull returns ErrUnexpectedEOF.
|
||||
return dst[:n], err
|
||||
}
|
||||
dst = dst[:n]
|
||||
if err == io.ErrUnexpectedEOF {
|
||||
return dst, io.EOF
|
||||
}
|
||||
}
|
||||
// Read until next line.
|
||||
in, err := r.buf.ReadBytes('\n')
|
||||
dst = append(dst, in...)
|
||||
return dst, err
|
||||
}
|
||||
|
||||
// jsonSplitSize is the size of each block.
|
||||
// Blocks will read this much and find the first following newline.
|
||||
// 128KB appears to be a very reasonable default.
|
||||
const jsonSplitSize = 128 << 10
|
||||
|
||||
// startReaders will read the header if needed and spin up a parser
|
||||
// and a number of workers based on GOMAXPROCS.
|
||||
// If an error is returned no goroutines have been started and r.err will have been set.
|
||||
func (r *PReader) startReaders() {
|
||||
r.bufferPool.New = func() interface{} {
|
||||
return make([]byte, jsonSplitSize+1024)
|
||||
}
|
||||
|
||||
// Create queue
|
||||
r.queue = make(chan *queueItem, runtime.GOMAXPROCS(0))
|
||||
r.input = make(chan *queueItem, runtime.GOMAXPROCS(0))
|
||||
r.readerWg.Add(1)
|
||||
|
||||
// Start splitter
|
||||
go func() {
|
||||
defer close(r.input)
|
||||
defer close(r.queue)
|
||||
defer r.readerWg.Done()
|
||||
for {
|
||||
next, err := r.nextSplit(jsonSplitSize, r.bufferPool.Get().([]byte))
|
||||
q := queueItem{
|
||||
input: next,
|
||||
dst: make(chan []jstream.KVS, 1),
|
||||
err: err,
|
||||
}
|
||||
select {
|
||||
case <-r.close:
|
||||
return
|
||||
case r.queue <- &q:
|
||||
}
|
||||
|
||||
select {
|
||||
case <-r.close:
|
||||
return
|
||||
case r.input <- &q:
|
||||
}
|
||||
if err != nil {
|
||||
// Exit on any error.
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// Start parsers
|
||||
for i := 0; i < runtime.GOMAXPROCS(0); i++ {
|
||||
go func() {
|
||||
for in := range r.input {
|
||||
if len(in.input) == 0 {
|
||||
in.dst <- nil
|
||||
continue
|
||||
}
|
||||
dst, ok := r.kvDstPool.Get().([]jstream.KVS)
|
||||
if !ok {
|
||||
dst = make([]jstream.KVS, 0, 1000)
|
||||
}
|
||||
|
||||
d := jstream.NewDecoder(bytes.NewBuffer(in.input), 0).ObjectAsKVS()
|
||||
stream := d.Stream()
|
||||
all := dst[:0]
|
||||
for mv := range stream {
|
||||
var kvs jstream.KVS
|
||||
if mv.ValueType == jstream.Object {
|
||||
// This is a JSON object type (that preserves key
|
||||
// order)
|
||||
kvs = mv.Value.(jstream.KVS)
|
||||
} else {
|
||||
// To be AWS S3 compatible Select for JSON needs to
|
||||
// output non-object JSON as single column value
|
||||
// i.e. a map with `_1` as key and value as the
|
||||
// non-object.
|
||||
kvs = jstream.KVS{jstream.KV{Key: "_1", Value: mv.Value}}
|
||||
}
|
||||
all = append(all, kvs)
|
||||
}
|
||||
// We don't need the input any more.
|
||||
//lint:ignore SA6002 Using pointer would allocate more since we would have to copy slice header before taking a pointer.
|
||||
r.bufferPool.Put(in.input)
|
||||
in.input = nil
|
||||
in.err = d.Err()
|
||||
in.dst <- all
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
// NewPReader - creates new parallel JSON reader using readCloser.
|
||||
// Should only be used for LINES types.
|
||||
func NewPReader(readCloser io.ReadCloser, args *ReaderArgs) *PReader {
|
||||
r := &PReader{
|
||||
args: args,
|
||||
buf: bufio.NewReaderSize(readCloser, jsonSplitSize*2),
|
||||
readCloser: readCloser,
|
||||
close: make(chan struct{}),
|
||||
}
|
||||
r.startReaders()
|
||||
return r
|
||||
}
|
||||
107
internal/s3select/json/preader_test.go
Normal file
107
internal/s3select/json/preader_test.go
Normal file
@@ -0,0 +1,107 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package json
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
)
|
||||
|
||||
func TestNewPReader(t *testing.T) {
|
||||
files, err := ioutil.ReadDir("testdata")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for _, file := range files {
|
||||
t.Run(file.Name(), func(t *testing.T) {
|
||||
f, err := os.Open(filepath.Join("testdata", file.Name()))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
r := NewPReader(f, &ReaderArgs{})
|
||||
var record sql.Record
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
r.Close()
|
||||
if err != io.EOF {
|
||||
t.Fatalf("Reading failed with %s, %s", err, file.Name())
|
||||
}
|
||||
})
|
||||
|
||||
t.Run(file.Name()+"-close", func(t *testing.T) {
|
||||
f, err := os.Open(filepath.Join("testdata", file.Name()))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
r := NewPReader(f, &ReaderArgs{})
|
||||
r.Close()
|
||||
var record sql.Record
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
if err != io.EOF {
|
||||
t.Fatalf("Reading failed with %s, %s", err, file.Name())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkPReader(b *testing.B) {
|
||||
files, err := ioutil.ReadDir("testdata")
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
for _, file := range files {
|
||||
b.Run(file.Name(), func(b *testing.B) {
|
||||
f, err := ioutil.ReadFile(filepath.Join("testdata", file.Name()))
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
b.SetBytes(int64(len(f)))
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
var record sql.Record
|
||||
for i := 0; i < b.N; i++ {
|
||||
r := NewPReader(ioutil.NopCloser(bytes.NewBuffer(f)), &ReaderArgs{})
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
r.Close()
|
||||
if err != io.EOF {
|
||||
b.Fatalf("Reading failed with %s, %s", err, file.Name())
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
134
internal/s3select/json/reader.go
Normal file
134
internal/s3select/json/reader.go
Normal file
@@ -0,0 +1,134 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package json
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"io"
|
||||
"sync"
|
||||
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
|
||||
"github.com/bcicen/jstream"
|
||||
)
|
||||
|
||||
// Reader - JSON record reader for S3Select.
|
||||
type Reader struct {
|
||||
args *ReaderArgs
|
||||
decoder *jstream.Decoder
|
||||
valueCh chan *jstream.MetaValue
|
||||
readCloser io.ReadCloser
|
||||
}
|
||||
|
||||
// Read - reads single record.
|
||||
func (r *Reader) Read(dst sql.Record) (sql.Record, error) {
|
||||
v, ok := <-r.valueCh
|
||||
if !ok {
|
||||
if err := r.decoder.Err(); err != nil {
|
||||
return nil, errJSONParsingError(err)
|
||||
}
|
||||
return nil, io.EOF
|
||||
}
|
||||
|
||||
var kvs jstream.KVS
|
||||
if v.ValueType == jstream.Object {
|
||||
// This is a JSON object type (that preserves key
|
||||
// order)
|
||||
kvs = v.Value.(jstream.KVS)
|
||||
} else {
|
||||
// To be AWS S3 compatible Select for JSON needs to
|
||||
// output non-object JSON as single column value
|
||||
// i.e. a map with `_1` as key and value as the
|
||||
// non-object.
|
||||
kvs = jstream.KVS{jstream.KV{Key: "_1", Value: v.Value}}
|
||||
}
|
||||
|
||||
dstRec, ok := dst.(*Record)
|
||||
if !ok {
|
||||
dstRec = &Record{}
|
||||
}
|
||||
dstRec.KVS = kvs
|
||||
dstRec.SelectFormat = sql.SelectFmtJSON
|
||||
return dstRec, nil
|
||||
}
|
||||
|
||||
// Close - closes underlying reader.
|
||||
func (r *Reader) Close() error {
|
||||
// Close the input.
|
||||
err := r.readCloser.Close()
|
||||
for range r.valueCh {
|
||||
// Drain values so we don't leak a goroutine.
|
||||
// Since we have closed the input, it should fail rather quickly.
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// NewReader - creates new JSON reader using readCloser.
|
||||
func NewReader(readCloser io.ReadCloser, args *ReaderArgs) *Reader {
|
||||
readCloser = &syncReadCloser{rc: readCloser}
|
||||
d := jstream.NewDecoder(readCloser, 0).ObjectAsKVS()
|
||||
return &Reader{
|
||||
args: args,
|
||||
decoder: d,
|
||||
valueCh: d.Stream(),
|
||||
readCloser: readCloser,
|
||||
}
|
||||
}
|
||||
|
||||
// syncReadCloser will wrap a readcloser and make it safe to call Close
|
||||
// while reads are running.
|
||||
// All read errors are also postponed until Close is called and
|
||||
// io.EOF is returned instead.
|
||||
type syncReadCloser struct {
|
||||
rc io.ReadCloser
|
||||
errMu sync.Mutex
|
||||
err error
|
||||
}
|
||||
|
||||
func (pr *syncReadCloser) Read(p []byte) (n int, err error) {
|
||||
// This ensures that Close will block until Read has completed.
|
||||
// This allows another goroutine to close the reader.
|
||||
pr.errMu.Lock()
|
||||
defer pr.errMu.Unlock()
|
||||
if pr.err != nil {
|
||||
return 0, io.EOF
|
||||
}
|
||||
n, pr.err = pr.rc.Read(p)
|
||||
if pr.err != nil {
|
||||
// Translate any error into io.EOF, so we don't crash:
|
||||
// https://github.com/bcicen/jstream/blob/master/scanner.go#L48
|
||||
return n, io.EOF
|
||||
}
|
||||
|
||||
return n, nil
|
||||
}
|
||||
|
||||
var errClosed = errors.New("read after close")
|
||||
|
||||
func (pr *syncReadCloser) Close() error {
|
||||
pr.errMu.Lock()
|
||||
defer pr.errMu.Unlock()
|
||||
if pr.err == errClosed {
|
||||
return nil
|
||||
}
|
||||
if pr.err != nil {
|
||||
return pr.err
|
||||
}
|
||||
pr.err = errClosed
|
||||
return pr.rc.Close()
|
||||
}
|
||||
107
internal/s3select/json/reader_test.go
Normal file
107
internal/s3select/json/reader_test.go
Normal file
@@ -0,0 +1,107 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package json
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
)
|
||||
|
||||
func TestNewReader(t *testing.T) {
|
||||
files, err := ioutil.ReadDir("testdata")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for _, file := range files {
|
||||
t.Run(file.Name(), func(t *testing.T) {
|
||||
f, err := os.Open(filepath.Join("testdata", file.Name()))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
r := NewReader(f, &ReaderArgs{})
|
||||
var record sql.Record
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
r.Close()
|
||||
if err != io.EOF {
|
||||
t.Fatalf("Reading failed with %s, %s", err, file.Name())
|
||||
}
|
||||
})
|
||||
|
||||
t.Run(file.Name()+"-close", func(t *testing.T) {
|
||||
f, err := os.Open(filepath.Join("testdata", file.Name()))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
r := NewReader(f, &ReaderArgs{})
|
||||
r.Close()
|
||||
var record sql.Record
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
if err != io.EOF {
|
||||
t.Fatalf("Reading failed with %s, %s", err, file.Name())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkReader(b *testing.B) {
|
||||
files, err := ioutil.ReadDir("testdata")
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
for _, file := range files {
|
||||
b.Run(file.Name(), func(b *testing.B) {
|
||||
f, err := ioutil.ReadFile(filepath.Join("testdata", file.Name()))
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
b.SetBytes(int64(len(f)))
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
var record sql.Record
|
||||
for i := 0; i < b.N; i++ {
|
||||
r := NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &ReaderArgs{})
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
r.Close()
|
||||
if err != io.EOF {
|
||||
b.Fatalf("Reading failed with %s, %s", err, file.Name())
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
206
internal/s3select/json/record.go
Normal file
206
internal/s3select/json/record.go
Normal file
@@ -0,0 +1,206 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package json
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/bcicen/jstream"
|
||||
csv "github.com/minio/csvparser"
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
)
|
||||
|
||||
// RawJSON is a byte-slice that contains valid JSON
|
||||
type RawJSON []byte
|
||||
|
||||
// MarshalJSON instance for []byte that assumes that byte-slice is
|
||||
// already serialized JSON
|
||||
func (b RawJSON) MarshalJSON() ([]byte, error) {
|
||||
return b, nil
|
||||
}
|
||||
|
||||
// Record - is JSON record.
|
||||
type Record struct {
|
||||
// Used in Set(), Marshal*()
|
||||
KVS jstream.KVS
|
||||
|
||||
SelectFormat sql.SelectObjectFormat
|
||||
}
|
||||
|
||||
// Get - gets the value for a column name.
|
||||
func (r *Record) Get(name string) (*sql.Value, error) {
|
||||
// Get is implemented directly in the sql package.
|
||||
return nil, errors.New("not implemented here")
|
||||
}
|
||||
|
||||
// Reset the record.
|
||||
func (r *Record) Reset() {
|
||||
if len(r.KVS) > 0 {
|
||||
r.KVS = r.KVS[:0]
|
||||
}
|
||||
}
|
||||
|
||||
// Clone the record and if possible use the destination provided.
|
||||
func (r *Record) Clone(dst sql.Record) sql.Record {
|
||||
other, ok := dst.(*Record)
|
||||
if !ok {
|
||||
other = &Record{}
|
||||
}
|
||||
if len(other.KVS) > 0 {
|
||||
other.KVS = other.KVS[:0]
|
||||
}
|
||||
other.KVS = append(other.KVS, r.KVS...)
|
||||
return other
|
||||
}
|
||||
|
||||
// Set - sets the value for a column name.
|
||||
func (r *Record) Set(name string, value *sql.Value) (sql.Record, error) {
|
||||
var v interface{}
|
||||
if b, ok := value.ToBool(); ok {
|
||||
v = b
|
||||
} else if f, ok := value.ToFloat(); ok {
|
||||
v = f
|
||||
} else if i, ok := value.ToInt(); ok {
|
||||
v = i
|
||||
} else if t, ok := value.ToTimestamp(); ok {
|
||||
v = sql.FormatSQLTimestamp(t)
|
||||
} else if s, ok := value.ToString(); ok {
|
||||
v = s
|
||||
} else if value.IsNull() {
|
||||
v = nil
|
||||
} else if b, ok := value.ToBytes(); ok {
|
||||
// This can either be raw json or a CSV value.
|
||||
// Only treat objects and arrays as JSON.
|
||||
if len(b) > 0 && (b[0] == '{' || b[0] == '[') {
|
||||
v = RawJSON(b)
|
||||
} else {
|
||||
v = string(b)
|
||||
}
|
||||
} else if arr, ok := value.ToArray(); ok {
|
||||
v = arr
|
||||
} else {
|
||||
return nil, fmt.Errorf("unsupported sql value %v and type %v", value, value.GetTypeString())
|
||||
}
|
||||
|
||||
name = strings.Replace(name, "*", "__ALL__", -1)
|
||||
r.KVS = append(r.KVS, jstream.KV{Key: name, Value: v})
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// WriteCSV - encodes to CSV data.
|
||||
func (r *Record) WriteCSV(writer io.Writer, opts sql.WriteCSVOpts) error {
|
||||
var csvRecord []string
|
||||
for _, kv := range r.KVS {
|
||||
var columnValue string
|
||||
switch val := kv.Value.(type) {
|
||||
case float64:
|
||||
columnValue = jsonFloat(val)
|
||||
case string:
|
||||
columnValue = val
|
||||
case bool, int64:
|
||||
columnValue = fmt.Sprintf("%v", val)
|
||||
case nil:
|
||||
columnValue = ""
|
||||
case RawJSON:
|
||||
columnValue = string([]byte(val))
|
||||
case []interface{}:
|
||||
b, err := json.Marshal(val)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
columnValue = string(b)
|
||||
default:
|
||||
return fmt.Errorf("Cannot marshal unhandled type: %T", kv.Value)
|
||||
}
|
||||
csvRecord = append(csvRecord, columnValue)
|
||||
}
|
||||
|
||||
w := csv.NewWriter(writer)
|
||||
w.Comma = opts.FieldDelimiter
|
||||
w.Quote = opts.Quote
|
||||
w.AlwaysQuote = opts.AlwaysQuote
|
||||
w.QuoteEscape = opts.QuoteEscape
|
||||
if err := w.Write(csvRecord); err != nil {
|
||||
return err
|
||||
}
|
||||
w.Flush()
|
||||
return w.Error()
|
||||
}
|
||||
|
||||
// Raw - returns the underlying representation.
|
||||
func (r *Record) Raw() (sql.SelectObjectFormat, interface{}) {
|
||||
return r.SelectFormat, r.KVS
|
||||
}
|
||||
|
||||
// WriteJSON - encodes to JSON data.
|
||||
func (r *Record) WriteJSON(writer io.Writer) error {
|
||||
return json.NewEncoder(writer).Encode(r.KVS)
|
||||
}
|
||||
|
||||
// Replace the underlying buffer of json data.
|
||||
func (r *Record) Replace(k interface{}) error {
|
||||
v, ok := k.(jstream.KVS)
|
||||
if !ok {
|
||||
return fmt.Errorf("cannot replace internal data in json record with type %T", k)
|
||||
}
|
||||
r.KVS = v
|
||||
return nil
|
||||
}
|
||||
|
||||
// NewRecord - creates new empty JSON record.
|
||||
func NewRecord(f sql.SelectObjectFormat) *Record {
|
||||
return &Record{
|
||||
KVS: jstream.KVS{},
|
||||
SelectFormat: f,
|
||||
}
|
||||
}
|
||||
|
||||
// jsonFloat converts a float to string similar to Go stdlib formats json floats.
|
||||
func jsonFloat(f float64) string {
|
||||
var tmp [32]byte
|
||||
dst := tmp[:0]
|
||||
|
||||
// Convert as if by ES6 number to string conversion.
|
||||
// This matches most other JSON generators.
|
||||
// See golang.org/issue/6384 and golang.org/issue/14135.
|
||||
// Like fmt %g, but the exponent cutoffs are different
|
||||
// and exponents themselves are not padded to two digits.
|
||||
abs := math.Abs(f)
|
||||
fmt := byte('f')
|
||||
if abs != 0 {
|
||||
if abs < 1e-6 || abs >= 1e21 {
|
||||
fmt = 'e'
|
||||
}
|
||||
}
|
||||
dst = strconv.AppendFloat(dst, f, fmt, -1, 64)
|
||||
if fmt == 'e' {
|
||||
// clean up e-09 to e-9
|
||||
n := len(dst)
|
||||
if n >= 4 && dst[n-4] == 'e' && dst[n-3] == '-' && dst[n-2] == '0' {
|
||||
dst[n-2] = dst[n-1]
|
||||
dst = dst[:n-1]
|
||||
}
|
||||
}
|
||||
return string(dst)
|
||||
}
|
||||
12
internal/s3select/json/testdata/10.json
vendored
Normal file
12
internal/s3select/json/testdata/10.json
vendored
Normal file
@@ -0,0 +1,12 @@
|
||||
[
|
||||
{
|
||||
"key_1": "value",
|
||||
"key_2": "value"
|
||||
}
|
||||
]
|
||||
[
|
||||
{
|
||||
"key_1": "value2",
|
||||
"key_2": "value3"
|
||||
}
|
||||
]
|
||||
8
internal/s3select/json/testdata/11.json
vendored
Normal file
8
internal/s3select/json/testdata/11.json
vendored
Normal file
@@ -0,0 +1,8 @@
|
||||
"a"
|
||||
1
|
||||
3.145
|
||||
["a"]
|
||||
{}
|
||||
{
|
||||
"a": 1
|
||||
}
|
||||
5
internal/s3select/json/testdata/12.json
vendored
Normal file
5
internal/s3select/json/testdata/12.json
vendored
Normal file
@@ -0,0 +1,5 @@
|
||||
{
|
||||
"a": 1
|
||||
}{
|
||||
"b": 2
|
||||
}
|
||||
1
internal/s3select/json/testdata/2.json
vendored
Normal file
1
internal/s3select/json/testdata/2.json
vendored
Normal file
@@ -0,0 +1 @@
|
||||
{"text": "hello world\\n2nd line"}
|
||||
1
internal/s3select/json/testdata/3.json
vendored
Normal file
1
internal/s3select/json/testdata/3.json
vendored
Normal file
@@ -0,0 +1 @@
|
||||
{"hello":"wor{l}d"}
|
||||
26
internal/s3select/json/testdata/4.json
vendored
Normal file
26
internal/s3select/json/testdata/4.json
vendored
Normal file
@@ -0,0 +1,26 @@
|
||||
{
|
||||
"id": "0001",
|
||||
"type": "donut",
|
||||
"name": "Cake",
|
||||
"ppu": 0.55,
|
||||
"batters":
|
||||
{
|
||||
"batter":
|
||||
[
|
||||
{ "id": "1001", "type": "Regular" },
|
||||
{ "id": "1002", "type": "Chocolate" },
|
||||
{ "id": "1003", "type": "Blueberry" },
|
||||
{ "id": "1004", "type": "Devil's Food" }
|
||||
]
|
||||
},
|
||||
"topping":
|
||||
[
|
||||
{ "id": "5001", "type": "None" },
|
||||
{ "id": "5002", "type": "Glazed" },
|
||||
{ "id": "5005", "type": "Sugar" },
|
||||
{ "id": "5007", "type": "Powdered Sugar" },
|
||||
{ "id": "5006", "type": "Chocolate with Sprinkles" },
|
||||
{ "id": "5003", "type": "Chocolate" },
|
||||
{ "id": "5004", "type": "Maple" }
|
||||
]
|
||||
}
|
||||
5
internal/s3select/json/testdata/5.json
vendored
Normal file
5
internal/s3select/json/testdata/5.json
vendored
Normal file
@@ -0,0 +1,5 @@
|
||||
{
|
||||
"foo": {
|
||||
"bar": "baz"
|
||||
}
|
||||
}
|
||||
1
internal/s3select/json/testdata/6.json
vendored
Normal file
1
internal/s3select/json/testdata/6.json
vendored
Normal file
@@ -0,0 +1 @@
|
||||
{ "name": "John", "age":28, "hobby": { "name": "chess", "type": "boardgame" }}
|
||||
3
internal/s3select/json/testdata/7.json
vendored
Normal file
3
internal/s3select/json/testdata/7.json
vendored
Normal file
@@ -0,0 +1,3 @@
|
||||
{"name":"Michael", "age": 31}
|
||||
{"name":"Andy", "age": 30}
|
||||
{"name":"Justin", "age": 19}
|
||||
2
internal/s3select/json/testdata/8.json
vendored
Normal file
2
internal/s3select/json/testdata/8.json
vendored
Normal file
@@ -0,0 +1,2 @@
|
||||
{"a":"}"
|
||||
}
|
||||
6
internal/s3select/json/testdata/9.json
vendored
Normal file
6
internal/s3select/json/testdata/9.json
vendored
Normal file
@@ -0,0 +1,6 @@
|
||||
[
|
||||
{
|
||||
"key_1": "value",
|
||||
"key_2": "value"
|
||||
}
|
||||
]
|
||||
Reference in New Issue
Block a user