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:
201
internal/s3select/csv/args.go
Normal file
201
internal/s3select/csv/args.go
Normal file
@@ -0,0 +1,201 @@
|
||||
// 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 csv
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"strings"
|
||||
"unicode/utf8"
|
||||
)
|
||||
|
||||
const (
|
||||
none = "none"
|
||||
use = "use"
|
||||
|
||||
defaultRecordDelimiter = "\n"
|
||||
defaultFieldDelimiter = ","
|
||||
defaultQuoteCharacter = `"`
|
||||
defaultQuoteEscapeCharacter = `"`
|
||||
defaultCommentCharacter = "#"
|
||||
|
||||
asneeded = "asneeded"
|
||||
)
|
||||
|
||||
// ReaderArgs - represents elements inside <InputSerialization><CSV> in request XML.
|
||||
type ReaderArgs struct {
|
||||
FileHeaderInfo string `xml:"FileHeaderInfo"`
|
||||
RecordDelimiter string `xml:"RecordDelimiter"`
|
||||
FieldDelimiter string `xml:"FieldDelimiter"`
|
||||
QuoteCharacter string `xml:"QuoteCharacter"`
|
||||
QuoteEscapeCharacter string `xml:"QuoteEscapeCharacter"`
|
||||
CommentCharacter string `xml:"Comments"`
|
||||
AllowQuotedRecordDelimiter bool `xml:"AllowQuotedRecordDelimiter"`
|
||||
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) (err error) {
|
||||
args.FileHeaderInfo = none
|
||||
args.RecordDelimiter = defaultRecordDelimiter
|
||||
args.FieldDelimiter = defaultFieldDelimiter
|
||||
args.QuoteCharacter = defaultQuoteCharacter
|
||||
args.QuoteEscapeCharacter = defaultQuoteEscapeCharacter
|
||||
args.CommentCharacter = defaultCommentCharacter
|
||||
args.AllowQuotedRecordDelimiter = false
|
||||
|
||||
for {
|
||||
// Read tokens from the XML document in a stream.
|
||||
t, err := d.Token()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
switch se := t.(type) {
|
||||
case xml.StartElement:
|
||||
tagName := se.Name.Local
|
||||
switch tagName {
|
||||
case "AllowQuotedRecordDelimiter":
|
||||
var b bool
|
||||
if err = d.DecodeElement(&b, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
args.AllowQuotedRecordDelimiter = b
|
||||
default:
|
||||
var s string
|
||||
if err = d.DecodeElement(&s, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
switch tagName {
|
||||
case "FileHeaderInfo":
|
||||
args.FileHeaderInfo = strings.ToLower(s)
|
||||
case "RecordDelimiter":
|
||||
args.RecordDelimiter = s
|
||||
case "FieldDelimiter":
|
||||
args.FieldDelimiter = s
|
||||
case "QuoteCharacter":
|
||||
if utf8.RuneCountInString(s) > 1 {
|
||||
return fmt.Errorf("unsupported QuoteCharacter '%v'", s)
|
||||
}
|
||||
args.QuoteCharacter = s
|
||||
case "QuoteEscapeCharacter":
|
||||
switch utf8.RuneCountInString(s) {
|
||||
case 0:
|
||||
args.QuoteEscapeCharacter = defaultQuoteEscapeCharacter
|
||||
case 1:
|
||||
args.QuoteEscapeCharacter = s
|
||||
default:
|
||||
return fmt.Errorf("unsupported QuoteEscapeCharacter '%v'", s)
|
||||
}
|
||||
case "Comments":
|
||||
args.CommentCharacter = s
|
||||
default:
|
||||
return errors.New("unrecognized option")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
args.unmarshaled = true
|
||||
return nil
|
||||
}
|
||||
|
||||
// WriterArgs - represents elements inside <OutputSerialization><CSV/> in request XML.
|
||||
type WriterArgs struct {
|
||||
QuoteFields string `xml:"QuoteFields"`
|
||||
RecordDelimiter string `xml:"RecordDelimiter"`
|
||||
FieldDelimiter string `xml:"FieldDelimiter"`
|
||||
QuoteCharacter string `xml:"QuoteCharacter"`
|
||||
QuoteEscapeCharacter string `xml:"QuoteEscapeCharacter"`
|
||||
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 {
|
||||
|
||||
args.QuoteFields = asneeded
|
||||
args.RecordDelimiter = defaultRecordDelimiter
|
||||
args.FieldDelimiter = defaultFieldDelimiter
|
||||
args.QuoteCharacter = defaultQuoteCharacter
|
||||
args.QuoteEscapeCharacter = defaultQuoteCharacter
|
||||
|
||||
for {
|
||||
// Read tokens from the XML document in a stream.
|
||||
t, err := d.Token()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
switch se := t.(type) {
|
||||
case xml.StartElement:
|
||||
var s string
|
||||
if err = d.DecodeElement(&s, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
switch se.Name.Local {
|
||||
case "QuoteFields":
|
||||
args.QuoteFields = strings.ToLower(s)
|
||||
case "RecordDelimiter":
|
||||
args.RecordDelimiter = s
|
||||
case "FieldDelimiter":
|
||||
args.FieldDelimiter = s
|
||||
case "QuoteCharacter":
|
||||
switch utf8.RuneCountInString(s) {
|
||||
case 0:
|
||||
args.QuoteCharacter = "\x00"
|
||||
case 1:
|
||||
args.QuoteCharacter = s
|
||||
default:
|
||||
return fmt.Errorf("unsupported QuoteCharacter '%v'", s)
|
||||
}
|
||||
case "QuoteEscapeCharacter":
|
||||
switch utf8.RuneCountInString(s) {
|
||||
case 0:
|
||||
args.QuoteEscapeCharacter = defaultQuoteEscapeCharacter
|
||||
case 1:
|
||||
args.QuoteEscapeCharacter = s
|
||||
default:
|
||||
return fmt.Errorf("unsupported QuoteCharacter '%v'", s)
|
||||
}
|
||||
default:
|
||||
return errors.New("unrecognized option")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
args.unmarshaled = true
|
||||
return nil
|
||||
}
|
||||
65
internal/s3select/csv/errors.go
Normal file
65
internal/s3select/csv/errors.go
Normal file
@@ -0,0 +1,65 @@
|
||||
// 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 csv
|
||||
|
||||
import "errors"
|
||||
|
||||
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 errCSVParsingError(err error) *s3Error {
|
||||
return &s3Error{
|
||||
code: "CSVParsingError",
|
||||
message: "Encountered an error parsing the CSV file. Check the file and try again.",
|
||||
statusCode: 400,
|
||||
cause: err,
|
||||
}
|
||||
}
|
||||
|
||||
func errInvalidTextEncodingError() *s3Error {
|
||||
return &s3Error{
|
||||
code: "InvalidTextEncoding",
|
||||
message: "UTF-8 encoding is required.",
|
||||
statusCode: 400,
|
||||
cause: errors.New("invalid utf8 encoding"),
|
||||
}
|
||||
}
|
||||
326
internal/s3select/csv/reader.go
Normal file
326
internal/s3select/csv/reader.go
Normal file
@@ -0,0 +1,326 @@
|
||||
// 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 csv
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"bytes"
|
||||
"fmt"
|
||||
"io"
|
||||
"runtime"
|
||||
"sync"
|
||||
"unicode/utf8"
|
||||
|
||||
csv "github.com/minio/csvparser"
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
)
|
||||
|
||||
// Reader - CSV record reader for S3Select.
|
||||
type Reader struct {
|
||||
args *ReaderArgs
|
||||
readCloser io.ReadCloser // raw input
|
||||
buf *bufio.Reader // input to the splitter
|
||||
columnNames []string // names of columns
|
||||
nameIndexMap map[string]int64 // name to column index
|
||||
current [][]string // 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
|
||||
csvDstPool sync.Pool // pool of [][]string 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 [][]string // 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 *Reader) 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.csvDstPool.Put(r.current)
|
||||
r.current = <-item.dst
|
||||
r.err = item.err
|
||||
r.recordsRead = 0
|
||||
}
|
||||
csvRecord := r.current[r.recordsRead]
|
||||
r.recordsRead++
|
||||
|
||||
// If no column names are set, use _(index)
|
||||
if r.columnNames == nil {
|
||||
r.columnNames = make([]string, len(csvRecord))
|
||||
for i := range csvRecord {
|
||||
r.columnNames[i] = fmt.Sprintf("_%v", i+1)
|
||||
}
|
||||
}
|
||||
|
||||
// If no index map, add that.
|
||||
if r.nameIndexMap == nil {
|
||||
r.nameIndexMap = make(map[string]int64)
|
||||
for i := range r.columnNames {
|
||||
r.nameIndexMap[r.columnNames[i]] = int64(i)
|
||||
}
|
||||
}
|
||||
dstRec, ok := dst.(*Record)
|
||||
if !ok {
|
||||
dstRec = &Record{}
|
||||
}
|
||||
dstRec.columnNames = r.columnNames
|
||||
dstRec.csvRecord = csvRecord
|
||||
dstRec.nameIndexMap = r.nameIndexMap
|
||||
|
||||
return dstRec, nil
|
||||
}
|
||||
|
||||
// Close - closes underlying reader.
|
||||
func (r *Reader) 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 *Reader) 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
|
||||
}
|
||||
|
||||
// csvSplitSize 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 csvSplitSize = 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 *Reader) startReaders(newReader func(io.Reader) *csv.Reader) error {
|
||||
if r.args.FileHeaderInfo != none {
|
||||
// Read column names
|
||||
// Get one line.
|
||||
b, err := r.nextSplit(0, nil)
|
||||
if err != nil {
|
||||
r.err = err
|
||||
return err
|
||||
}
|
||||
if !utf8.Valid(b) {
|
||||
return errInvalidTextEncodingError()
|
||||
}
|
||||
reader := newReader(bytes.NewReader(b))
|
||||
record, err := reader.Read()
|
||||
if err != nil {
|
||||
r.err = err
|
||||
if err != io.EOF {
|
||||
r.err = errCSVParsingError(err)
|
||||
return errCSVParsingError(err)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
if r.args.FileHeaderInfo == use {
|
||||
// Copy column names since records will be reused.
|
||||
columns := append(make([]string, 0, len(record)), record...)
|
||||
r.columnNames = columns
|
||||
}
|
||||
}
|
||||
|
||||
r.bufferPool.New = func() interface{} {
|
||||
return make([]byte, csvSplitSize+1024)
|
||||
}
|
||||
|
||||
// Return first block
|
||||
next, nextErr := r.nextSplit(csvSplitSize, r.bufferPool.Get().([]byte))
|
||||
// Check if first block is valid.
|
||||
if !utf8.Valid(next) {
|
||||
return errInvalidTextEncodingError()
|
||||
}
|
||||
|
||||
// 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 {
|
||||
q := queueItem{
|
||||
input: next,
|
||||
dst: make(chan [][]string, 1),
|
||||
err: nextErr,
|
||||
}
|
||||
select {
|
||||
case <-r.close:
|
||||
return
|
||||
case r.queue <- &q:
|
||||
}
|
||||
|
||||
select {
|
||||
case <-r.close:
|
||||
return
|
||||
case r.input <- &q:
|
||||
}
|
||||
if nextErr != nil {
|
||||
// Exit on any error.
|
||||
return
|
||||
}
|
||||
next, nextErr = r.nextSplit(csvSplitSize, r.bufferPool.Get().([]byte))
|
||||
}
|
||||
}()
|
||||
|
||||
// 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.csvDstPool.Get().([][]string)
|
||||
if !ok {
|
||||
dst = make([][]string, 0, 1000)
|
||||
}
|
||||
|
||||
cr := newReader(bytes.NewBuffer(in.input))
|
||||
all := dst[:0]
|
||||
err := func() error {
|
||||
// Read all records until EOF or another error.
|
||||
for {
|
||||
record, err := cr.Read()
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return errCSVParsingError(err)
|
||||
}
|
||||
var recDst []string
|
||||
if len(dst) > len(all) {
|
||||
recDst = dst[len(all)]
|
||||
}
|
||||
if cap(recDst) < len(record) {
|
||||
recDst = make([]string, len(record))
|
||||
}
|
||||
recDst = recDst[:len(record)]
|
||||
copy(recDst, record)
|
||||
all = append(all, recDst)
|
||||
}
|
||||
}()
|
||||
if err != nil {
|
||||
in.err = err
|
||||
}
|
||||
// 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.dst <- all
|
||||
}
|
||||
}()
|
||||
}
|
||||
return nil
|
||||
|
||||
}
|
||||
|
||||
// NewReader - creates new CSV reader using readCloser.
|
||||
func NewReader(readCloser io.ReadCloser, args *ReaderArgs) (*Reader, error) {
|
||||
if args == nil || args.IsEmpty() {
|
||||
panic(fmt.Errorf("empty args passed %v", args))
|
||||
}
|
||||
csvIn := io.Reader(readCloser)
|
||||
if args.RecordDelimiter != "\n" {
|
||||
csvIn = &recordTransform{
|
||||
reader: readCloser,
|
||||
recordDelimiter: []byte(args.RecordDelimiter),
|
||||
oneByte: make([]byte, len(args.RecordDelimiter)-1),
|
||||
}
|
||||
}
|
||||
|
||||
r := &Reader{
|
||||
args: args,
|
||||
buf: bufio.NewReaderSize(csvIn, csvSplitSize*2),
|
||||
readCloser: readCloser,
|
||||
close: make(chan struct{}),
|
||||
}
|
||||
|
||||
// Assume args are validated by ReaderArgs.UnmarshalXML()
|
||||
newCsvReader := func(r io.Reader) *csv.Reader {
|
||||
ret := csv.NewReader(r)
|
||||
ret.Comma = []rune(args.FieldDelimiter)[0]
|
||||
ret.Comment = []rune(args.CommentCharacter)[0]
|
||||
ret.Quote = []rune{}
|
||||
if len([]rune(args.QuoteCharacter)) > 0 {
|
||||
// Add the first rune of args.QuoteChracter
|
||||
ret.Quote = append(ret.Quote, []rune(args.QuoteCharacter)[0])
|
||||
}
|
||||
ret.QuoteEscape = []rune(args.QuoteEscapeCharacter)[0]
|
||||
ret.FieldsPerRecord = -1
|
||||
// If LazyQuotes is true, a quote may appear in an unquoted field and a
|
||||
// non-doubled quote may appear in a quoted field.
|
||||
ret.LazyQuotes = true
|
||||
// We do not trim leading space to keep consistent with s3.
|
||||
ret.TrimLeadingSpace = false
|
||||
ret.ReuseRecord = true
|
||||
return ret
|
||||
}
|
||||
|
||||
return r, r.startReaders(newCsvReader)
|
||||
}
|
||||
650
internal/s3select/csv/reader_contrib_test.go
Normal file
650
internal/s3select/csv/reader_contrib_test.go
Normal file
@@ -0,0 +1,650 @@
|
||||
/*
|
||||
* MinIO Object Storage (c) 2021 MinIO, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package csv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"path/filepath"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/klauspost/compress/zip"
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
)
|
||||
|
||||
func TestRead(t *testing.T) {
|
||||
cases := []struct {
|
||||
content string
|
||||
recordDelimiter string
|
||||
fieldDelimiter string
|
||||
}{
|
||||
{"1,2,3\na,b,c\n", "\n", ","},
|
||||
{"1,2,3\ta,b,c\t", "\t", ","},
|
||||
{"1,2,3\r\na,b,c\r\n", "\r\n", ","},
|
||||
}
|
||||
|
||||
for i, c := range cases {
|
||||
var err error
|
||||
var record sql.Record
|
||||
var result bytes.Buffer
|
||||
|
||||
r, _ := NewReader(ioutil.NopCloser(strings.NewReader(c.content)), &ReaderArgs{
|
||||
FileHeaderInfo: none,
|
||||
RecordDelimiter: c.recordDelimiter,
|
||||
FieldDelimiter: c.fieldDelimiter,
|
||||
QuoteCharacter: defaultQuoteCharacter,
|
||||
QuoteEscapeCharacter: defaultQuoteEscapeCharacter,
|
||||
CommentCharacter: defaultCommentCharacter,
|
||||
AllowQuotedRecordDelimiter: false,
|
||||
unmarshaled: true,
|
||||
})
|
||||
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
opts := sql.WriteCSVOpts{
|
||||
FieldDelimiter: []rune(c.fieldDelimiter)[0],
|
||||
Quote: '"',
|
||||
QuoteEscape: '"',
|
||||
AlwaysQuote: false,
|
||||
}
|
||||
record.WriteCSV(&result, opts)
|
||||
result.Truncate(result.Len() - 1)
|
||||
result.WriteString(c.recordDelimiter)
|
||||
}
|
||||
r.Close()
|
||||
if err != io.EOF {
|
||||
t.Fatalf("Case %d failed with %s", i, err)
|
||||
}
|
||||
|
||||
if result.String() != c.content {
|
||||
t.Errorf("Case %d failed: expected %v result %v", i, c.content, result.String())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type tester interface {
|
||||
Fatal(...interface{})
|
||||
}
|
||||
|
||||
func openTestFile(t tester, file string) []byte {
|
||||
f, err := ioutil.ReadFile(filepath.Join("testdata/testdata.zip"))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
z, err := zip.NewReader(bytes.NewReader(f), int64(len(f)))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for _, f := range z.File {
|
||||
if f.Name == file {
|
||||
rc, err := f.Open()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer rc.Close()
|
||||
b, err := ioutil.ReadAll(rc)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return b
|
||||
}
|
||||
}
|
||||
t.Fatal(file, "not found in testdata/testdata.zip")
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestReadExtended(t *testing.T) {
|
||||
cases := []struct {
|
||||
file string
|
||||
recordDelimiter string
|
||||
fieldDelimiter string
|
||||
header bool
|
||||
wantColumns []string
|
||||
wantTenFields string
|
||||
totalFields int
|
||||
}{
|
||||
{
|
||||
file: "nyc-taxi-data-100k.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
header: true,
|
||||
wantColumns: []string{"trip_id", "vendor_id", "pickup_datetime", "dropoff_datetime", "store_and_fwd_flag", "rate_code_id", "pickup_longitude", "pickup_latitude", "dropoff_longitude", "dropoff_latitude", "passenger_count", "trip_distance", "fare_amount", "extra", "mta_tax", "tip_amount", "tolls_amount", "ehail_fee", "improvement_surcharge", "total_amount", "payment_type", "trip_type", "pickup", "dropoff", "cab_type", "precipitation", "snow_depth", "snowfall", "max_temp", "min_temp", "wind", "pickup_nyct2010_gid", "pickup_ctlabel", "pickup_borocode", "pickup_boroname", "pickup_ct2010", "pickup_boroct2010", "pickup_cdeligibil", "pickup_ntacode", "pickup_ntaname", "pickup_puma", "dropoff_nyct2010_gid", "dropoff_ctlabel", "dropoff_borocode", "dropoff_boroname", "dropoff_ct2010", "dropoff_boroct2010", "dropoff_cdeligibil", "dropoff_ntacode", "dropoff_ntaname", "dropoff_puma"},
|
||||
wantTenFields: `3389224,2,2014-03-26 00:26:15,2014-03-26 00:28:38,N,1,-73.950431823730469,40.792251586914063,-73.938949584960937,40.794425964355469,1,0.84,4.5,0.5,0.5,1,0,,,6.5,1,1,75,74,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1828,180,1,Manhattan,018000,1018000,E,MN34,East Harlem North,3804
|
||||
3389225,2,2014-03-31 09:42:15,2014-03-31 10:01:17,N,1,-73.950340270996094,40.792228698730469,-73.941970825195313,40.842235565185547,1,4.47,17.5,0,0.5,0,0,,,18,2,1,75,244,green,0.16,0.0,0.0,56,36,8.28,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,911,251,1,Manhattan,025100,1025100,E,MN36,Washington Heights South,3801
|
||||
3389226,2,2014-03-26 17:13:28,2014-03-26 17:19:07,N,1,-73.949493408203125,40.793506622314453,-73.943374633789063,40.786155700683594,1,0.82,5.5,1,0.5,0,0,,,7,1,1,75,75,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1387,164,1,Manhattan,016400,1016400,E,MN33,East Harlem South,3804
|
||||
3389227,2,2014-03-14 21:07:19,2014-03-14 21:11:41,N,1,-73.950538635253906,40.792228698730469,-73.940811157226563,40.809253692626953,1,1.40,6,0.5,0.5,0,0,,,7,2,1,75,42,green,0.00,0.0,0.0,46,22,5.59,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1184,208,1,Manhattan,020800,1020800,E,MN03,Central Harlem North-Polo Grounds,3803
|
||||
3389228,1,2014-03-28 13:52:56,2014-03-28 14:29:01,N,1,-73.950569152832031,40.792312622070313,-73.868507385253906,40.688491821289063,2,16.10,46,0,0.5,0,5.33,,,51.83,2,,75,63,green,0.04,0.0,0.0,62,37,5.37,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1544,1182.02,3,Brooklyn,118202,3118202,E,BK83,Cypress Hills-City Line,4008
|
||||
3389229,2,2014-03-07 09:46:32,2014-03-07 09:55:01,N,1,-73.952301025390625,40.789798736572266,-73.935806274414062,40.794448852539063,1,1.67,8,0,0.5,2,0,,,10.5,1,1,75,74,green,0.00,3.9,0.0,37,26,7.83,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1553,178,1,Manhattan,017800,1017800,E,MN34,East Harlem North,3804
|
||||
3389230,2,2014-03-17 18:23:05,2014-03-17 18:28:38,N,1,-73.952346801757813,40.789844512939453,-73.946319580078125,40.783851623535156,5,0.95,5.5,1,0.5,0.65,0,,,7.65,1,1,75,263,green,0.00,0.0,0.0,35,23,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,32,156.01,1,Manhattan,015601,1015601,I,MN32,Yorkville,3805
|
||||
3389231,1,2014-03-19 19:09:36,2014-03-19 19:12:20,N,1,-73.952377319335938,40.789779663085938,-73.947494506835938,40.796474456787109,1,0.50,4,1,0.5,1,0,,,6.5,1,,75,75,green,0.92,0.0,0.0,46,32,7.16,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1401,174.02,1,Manhattan,017402,1017402,E,MN33,East Harlem South,3804
|
||||
3389232,2,2014-03-20 19:06:28,2014-03-20 19:21:35,N,1,-73.952583312988281,40.789516448974609,-73.985870361328125,40.776973724365234,2,3.04,13,1,0.5,2.8,0,,,17.3,1,1,75,143,green,0.00,0.0,0.0,54,40,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1742,155,1,Manhattan,015500,1015500,I,MN14,Lincoln Square,3806
|
||||
3389233,2,2014-03-29 09:38:12,2014-03-29 09:44:16,N,1,-73.952728271484375,40.789501190185547,-73.950935363769531,40.775600433349609,1,1.10,6.5,0,0.5,1.3,0,,,8.3,1,1,75,263,green,1.81,0.0,0.0,59,43,10.74,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,2048,138,1,Manhattan,013800,1013800,I,MN32,Yorkville,3805
|
||||
`,
|
||||
totalFields: 308*2 + 1,
|
||||
}, {
|
||||
file: "nyc-taxi-data-tabs-100k.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: "\t",
|
||||
header: true,
|
||||
wantColumns: []string{"trip_id", "vendor_id", "pickup_datetime", "dropoff_datetime", "store_and_fwd_flag", "rate_code_id", "pickup_longitude", "pickup_latitude", "dropoff_longitude", "dropoff_latitude", "passenger_count", "trip_distance", "fare_amount", "extra", "mta_tax", "tip_amount", "tolls_amount", "ehail_fee", "improvement_surcharge", "total_amount", "payment_type", "trip_type", "pickup", "dropoff", "cab_type", "precipitation", "snow_depth", "snowfall", "max_temp", "min_temp", "wind", "pickup_nyct2010_gid", "pickup_ctlabel", "pickup_borocode", "pickup_boroname", "pickup_ct2010", "pickup_boroct2010", "pickup_cdeligibil", "pickup_ntacode", "pickup_ntaname", "pickup_puma", "dropoff_nyct2010_gid", "dropoff_ctlabel", "dropoff_borocode", "dropoff_boroname", "dropoff_ct2010", "dropoff_boroct2010", "dropoff_cdeligibil", "dropoff_ntacode", "dropoff_ntaname", "dropoff_puma"},
|
||||
wantTenFields: `3389224,2,2014-03-26 00:26:15,2014-03-26 00:28:38,N,1,-73.950431823730469,40.792251586914063,-73.938949584960937,40.794425964355469,1,0.84,4.5,0.5,0.5,1,0,,,6.5,1,1,75,74,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1828,180,1,Manhattan,018000,1018000,E,MN34,East Harlem North,3804
|
||||
3389225,2,2014-03-31 09:42:15,2014-03-31 10:01:17,N,1,-73.950340270996094,40.792228698730469,-73.941970825195313,40.842235565185547,1,4.47,17.5,0,0.5,0,0,,,18,2,1,75,244,green,0.16,0.0,0.0,56,36,8.28,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,911,251,1,Manhattan,025100,1025100,E,MN36,Washington Heights South,3801
|
||||
3389226,2,2014-03-26 17:13:28,2014-03-26 17:19:07,N,1,-73.949493408203125,40.793506622314453,-73.943374633789063,40.786155700683594,1,0.82,5.5,1,0.5,0,0,,,7,1,1,75,75,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1387,164,1,Manhattan,016400,1016400,E,MN33,East Harlem South,3804
|
||||
3389227,2,2014-03-14 21:07:19,2014-03-14 21:11:41,N,1,-73.950538635253906,40.792228698730469,-73.940811157226563,40.809253692626953,1,1.40,6,0.5,0.5,0,0,,,7,2,1,75,42,green,0.00,0.0,0.0,46,22,5.59,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1184,208,1,Manhattan,020800,1020800,E,MN03,Central Harlem North-Polo Grounds,3803
|
||||
3389228,1,2014-03-28 13:52:56,2014-03-28 14:29:01,N,1,-73.950569152832031,40.792312622070313,-73.868507385253906,40.688491821289063,2,16.10,46,0,0.5,0,5.33,,,51.83,2,,75,63,green,0.04,0.0,0.0,62,37,5.37,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1544,1182.02,3,Brooklyn,118202,3118202,E,BK83,Cypress Hills-City Line,4008
|
||||
3389229,2,2014-03-07 09:46:32,2014-03-07 09:55:01,N,1,-73.952301025390625,40.789798736572266,-73.935806274414062,40.794448852539063,1,1.67,8,0,0.5,2,0,,,10.5,1,1,75,74,green,0.00,3.9,0.0,37,26,7.83,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1553,178,1,Manhattan,017800,1017800,E,MN34,East Harlem North,3804
|
||||
3389230,2,2014-03-17 18:23:05,2014-03-17 18:28:38,N,1,-73.952346801757813,40.789844512939453,-73.946319580078125,40.783851623535156,5,0.95,5.5,1,0.5,0.65,0,,,7.65,1,1,75,263,green,0.00,0.0,0.0,35,23,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,32,156.01,1,Manhattan,015601,1015601,I,MN32,Yorkville,3805
|
||||
3389231,1,2014-03-19 19:09:36,2014-03-19 19:12:20,N,1,-73.952377319335938,40.789779663085938,-73.947494506835938,40.796474456787109,1,0.50,4,1,0.5,1,0,,,6.5,1,,75,75,green,0.92,0.0,0.0,46,32,7.16,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1401,174.02,1,Manhattan,017402,1017402,E,MN33,East Harlem South,3804
|
||||
3389232,2,2014-03-20 19:06:28,2014-03-20 19:21:35,N,1,-73.952583312988281,40.789516448974609,-73.985870361328125,40.776973724365234,2,3.04,13,1,0.5,2.8,0,,,17.3,1,1,75,143,green,0.00,0.0,0.0,54,40,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1742,155,1,Manhattan,015500,1015500,I,MN14,Lincoln Square,3806
|
||||
3389233,2,2014-03-29 09:38:12,2014-03-29 09:44:16,N,1,-73.952728271484375,40.789501190185547,-73.950935363769531,40.775600433349609,1,1.10,6.5,0,0.5,1.3,0,,,8.3,1,1,75,263,green,1.81,0.0,0.0,59,43,10.74,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,2048,138,1,Manhattan,013800,1013800,I,MN32,Yorkville,3805
|
||||
`,
|
||||
totalFields: 308*2 + 1,
|
||||
}, {
|
||||
file: "nyc-taxi-data-100k-single-delim.csv",
|
||||
recordDelimiter: "^",
|
||||
fieldDelimiter: ",",
|
||||
header: true,
|
||||
wantColumns: []string{"trip_id", "vendor_id", "pickup_datetime", "dropoff_datetime", "store_and_fwd_flag", "rate_code_id", "pickup_longitude", "pickup_latitude", "dropoff_longitude", "dropoff_latitude", "passenger_count", "trip_distance", "fare_amount", "extra", "mta_tax", "tip_amount", "tolls_amount", "ehail_fee", "improvement_surcharge", "total_amount", "payment_type", "trip_type", "pickup", "dropoff", "cab_type", "precipitation", "snow_depth", "snowfall", "max_temp", "min_temp", "wind", "pickup_nyct2010_gid", "pickup_ctlabel", "pickup_borocode", "pickup_boroname", "pickup_ct2010", "pickup_boroct2010", "pickup_cdeligibil", "pickup_ntacode", "pickup_ntaname", "pickup_puma", "dropoff_nyct2010_gid", "dropoff_ctlabel", "dropoff_borocode", "dropoff_boroname", "dropoff_ct2010", "dropoff_boroct2010", "dropoff_cdeligibil", "dropoff_ntacode", "dropoff_ntaname", "dropoff_puma"},
|
||||
wantTenFields: `3389224,2,2014-03-26 00:26:15,2014-03-26 00:28:38,N,1,-73.950431823730469,40.792251586914063,-73.938949584960937,40.794425964355469,1,0.84,4.5,0.5,0.5,1,0,,,6.5,1,1,75,74,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1828,180,1,Manhattan,018000,1018000,E,MN34,East Harlem North,3804
|
||||
3389225,2,2014-03-31 09:42:15,2014-03-31 10:01:17,N,1,-73.950340270996094,40.792228698730469,-73.941970825195313,40.842235565185547,1,4.47,17.5,0,0.5,0,0,,,18,2,1,75,244,green,0.16,0.0,0.0,56,36,8.28,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,911,251,1,Manhattan,025100,1025100,E,MN36,Washington Heights South,3801
|
||||
3389226,2,2014-03-26 17:13:28,2014-03-26 17:19:07,N,1,-73.949493408203125,40.793506622314453,-73.943374633789063,40.786155700683594,1,0.82,5.5,1,0.5,0,0,,,7,1,1,75,75,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1387,164,1,Manhattan,016400,1016400,E,MN33,East Harlem South,3804
|
||||
3389227,2,2014-03-14 21:07:19,2014-03-14 21:11:41,N,1,-73.950538635253906,40.792228698730469,-73.940811157226563,40.809253692626953,1,1.40,6,0.5,0.5,0,0,,,7,2,1,75,42,green,0.00,0.0,0.0,46,22,5.59,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1184,208,1,Manhattan,020800,1020800,E,MN03,Central Harlem North-Polo Grounds,3803
|
||||
3389228,1,2014-03-28 13:52:56,2014-03-28 14:29:01,N,1,-73.950569152832031,40.792312622070313,-73.868507385253906,40.688491821289063,2,16.10,46,0,0.5,0,5.33,,,51.83,2,,75,63,green,0.04,0.0,0.0,62,37,5.37,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1544,1182.02,3,Brooklyn,118202,3118202,E,BK83,Cypress Hills-City Line,4008
|
||||
3389229,2,2014-03-07 09:46:32,2014-03-07 09:55:01,N,1,-73.952301025390625,40.789798736572266,-73.935806274414062,40.794448852539063,1,1.67,8,0,0.5,2,0,,,10.5,1,1,75,74,green,0.00,3.9,0.0,37,26,7.83,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1553,178,1,Manhattan,017800,1017800,E,MN34,East Harlem North,3804
|
||||
3389230,2,2014-03-17 18:23:05,2014-03-17 18:28:38,N,1,-73.952346801757813,40.789844512939453,-73.946319580078125,40.783851623535156,5,0.95,5.5,1,0.5,0.65,0,,,7.65,1,1,75,263,green,0.00,0.0,0.0,35,23,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,32,156.01,1,Manhattan,015601,1015601,I,MN32,Yorkville,3805
|
||||
3389231,1,2014-03-19 19:09:36,2014-03-19 19:12:20,N,1,-73.952377319335938,40.789779663085938,-73.947494506835938,40.796474456787109,1,0.50,4,1,0.5,1,0,,,6.5,1,,75,75,green,0.92,0.0,0.0,46,32,7.16,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1401,174.02,1,Manhattan,017402,1017402,E,MN33,East Harlem South,3804
|
||||
3389232,2,2014-03-20 19:06:28,2014-03-20 19:21:35,N,1,-73.952583312988281,40.789516448974609,-73.985870361328125,40.776973724365234,2,3.04,13,1,0.5,2.8,0,,,17.3,1,1,75,143,green,0.00,0.0,0.0,54,40,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1742,155,1,Manhattan,015500,1015500,I,MN14,Lincoln Square,3806
|
||||
3389233,2,2014-03-29 09:38:12,2014-03-29 09:44:16,N,1,-73.952728271484375,40.789501190185547,-73.950935363769531,40.775600433349609,1,1.10,6.5,0,0.5,1.3,0,,,8.3,1,1,75,263,green,1.81,0.0,0.0,59,43,10.74,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,2048,138,1,Manhattan,013800,1013800,I,MN32,Yorkville,3805
|
||||
`,
|
||||
totalFields: 308*2 + 1,
|
||||
}, {
|
||||
file: "nyc-taxi-data-100k-multi-delim.csv",
|
||||
recordDelimiter: "^Y",
|
||||
fieldDelimiter: ",",
|
||||
header: true,
|
||||
wantColumns: []string{"trip_id", "vendor_id", "pickup_datetime", "dropoff_datetime", "store_and_fwd_flag", "rate_code_id", "pickup_longitude", "pickup_latitude", "dropoff_longitude", "dropoff_latitude", "passenger_count", "trip_distance", "fare_amount", "extra", "mta_tax", "tip_amount", "tolls_amount", "ehail_fee", "improvement_surcharge", "total_amount", "payment_type", "trip_type", "pickup", "dropoff", "cab_type", "precipitation", "snow_depth", "snowfall", "max_temp", "min_temp", "wind", "pickup_nyct2010_gid", "pickup_ctlabel", "pickup_borocode", "pickup_boroname", "pickup_ct2010", "pickup_boroct2010", "pickup_cdeligibil", "pickup_ntacode", "pickup_ntaname", "pickup_puma", "dropoff_nyct2010_gid", "dropoff_ctlabel", "dropoff_borocode", "dropoff_boroname", "dropoff_ct2010", "dropoff_boroct2010", "dropoff_cdeligibil", "dropoff_ntacode", "dropoff_ntaname", "dropoff_puma"},
|
||||
wantTenFields: `3389224,2,2014-03-26 00:26:15,2014-03-26 00:28:38,N,1,-73.950431823730469,40.792251586914063,-73.938949584960937,40.794425964355469,1,0.84,4.5,0.5,0.5,1,0,,,6.5,1,1,75,74,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1828,180,1,Manhattan,018000,1018000,E,MN34,East Harlem North,3804
|
||||
3389225,2,2014-03-31 09:42:15,2014-03-31 10:01:17,N,1,-73.950340270996094,40.792228698730469,-73.941970825195313,40.842235565185547,1,4.47,17.5,0,0.5,0,0,,,18,2,1,75,244,green,0.16,0.0,0.0,56,36,8.28,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,911,251,1,Manhattan,025100,1025100,E,MN36,Washington Heights South,3801
|
||||
3389226,2,2014-03-26 17:13:28,2014-03-26 17:19:07,N,1,-73.949493408203125,40.793506622314453,-73.943374633789063,40.786155700683594,1,0.82,5.5,1,0.5,0,0,,,7,1,1,75,75,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1387,164,1,Manhattan,016400,1016400,E,MN33,East Harlem South,3804
|
||||
3389227,2,2014-03-14 21:07:19,2014-03-14 21:11:41,N,1,-73.950538635253906,40.792228698730469,-73.940811157226563,40.809253692626953,1,1.40,6,0.5,0.5,0,0,,,7,2,1,75,42,green,0.00,0.0,0.0,46,22,5.59,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1184,208,1,Manhattan,020800,1020800,E,MN03,Central Harlem North-Polo Grounds,3803
|
||||
3389228,1,2014-03-28 13:52:56,2014-03-28 14:29:01,N,1,-73.950569152832031,40.792312622070313,-73.868507385253906,40.688491821289063,2,16.10,46,0,0.5,0,5.33,,,51.83,2,,75,63,green,0.04,0.0,0.0,62,37,5.37,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1544,1182.02,3,Brooklyn,118202,3118202,E,BK83,Cypress Hills-City Line,4008
|
||||
3389229,2,2014-03-07 09:46:32,2014-03-07 09:55:01,N,1,-73.952301025390625,40.789798736572266,-73.935806274414062,40.794448852539063,1,1.67,8,0,0.5,2,0,,,10.5,1,1,75,74,green,0.00,3.9,0.0,37,26,7.83,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1553,178,1,Manhattan,017800,1017800,E,MN34,East Harlem North,3804
|
||||
3389230,2,2014-03-17 18:23:05,2014-03-17 18:28:38,N,1,-73.952346801757813,40.789844512939453,-73.946319580078125,40.783851623535156,5,0.95,5.5,1,0.5,0.65,0,,,7.65,1,1,75,263,green,0.00,0.0,0.0,35,23,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,32,156.01,1,Manhattan,015601,1015601,I,MN32,Yorkville,3805
|
||||
3389231,1,2014-03-19 19:09:36,2014-03-19 19:12:20,N,1,-73.952377319335938,40.789779663085938,-73.947494506835938,40.796474456787109,1,0.50,4,1,0.5,1,0,,,6.5,1,,75,75,green,0.92,0.0,0.0,46,32,7.16,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1401,174.02,1,Manhattan,017402,1017402,E,MN33,East Harlem South,3804
|
||||
3389232,2,2014-03-20 19:06:28,2014-03-20 19:21:35,N,1,-73.952583312988281,40.789516448974609,-73.985870361328125,40.776973724365234,2,3.04,13,1,0.5,2.8,0,,,17.3,1,1,75,143,green,0.00,0.0,0.0,54,40,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1742,155,1,Manhattan,015500,1015500,I,MN14,Lincoln Square,3806
|
||||
3389233,2,2014-03-29 09:38:12,2014-03-29 09:44:16,N,1,-73.952728271484375,40.789501190185547,-73.950935363769531,40.775600433349609,1,1.10,6.5,0,0.5,1.3,0,,,8.3,1,1,75,263,green,1.81,0.0,0.0,59,43,10.74,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,2048,138,1,Manhattan,013800,1013800,I,MN32,Yorkville,3805
|
||||
`,
|
||||
totalFields: 308*2 + 1,
|
||||
}, {
|
||||
file: "nyc-taxi-data-noheader-100k.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
header: false,
|
||||
wantColumns: []string{"_1", "_2", "_3", "_4", "_5", "_6", "_7", "_8", "_9", "_10", "_11", "_12", "_13", "_14", "_15", "_16", "_17", "_18", "_19", "_20", "_21", "_22", "_23", "_24", "_25", "_26", "_27", "_28", "_29", "_30", "_31", "_32", "_33", "_34", "_35", "_36", "_37", "_38", "_39", "_40", "_41", "_42", "_43", "_44", "_45", "_46", "_47", "_48", "_49", "_50", "_51"},
|
||||
wantTenFields: `3389224,2,2014-03-26 00:26:15,2014-03-26 00:28:38,N,1,-73.950431823730469,40.792251586914063,-73.938949584960937,40.794425964355469,1,0.84,4.5,0.5,0.5,1,0,,,6.5,1,1,75,74,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1828,180,1,Manhattan,018000,1018000,E,MN34,East Harlem North,3804
|
||||
3389225,2,2014-03-31 09:42:15,2014-03-31 10:01:17,N,1,-73.950340270996094,40.792228698730469,-73.941970825195313,40.842235565185547,1,4.47,17.5,0,0.5,0,0,,,18,2,1,75,244,green,0.16,0.0,0.0,56,36,8.28,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,911,251,1,Manhattan,025100,1025100,E,MN36,Washington Heights South,3801
|
||||
3389226,2,2014-03-26 17:13:28,2014-03-26 17:19:07,N,1,-73.949493408203125,40.793506622314453,-73.943374633789063,40.786155700683594,1,0.82,5.5,1,0.5,0,0,,,7,1,1,75,75,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1387,164,1,Manhattan,016400,1016400,E,MN33,East Harlem South,3804
|
||||
3389227,2,2014-03-14 21:07:19,2014-03-14 21:11:41,N,1,-73.950538635253906,40.792228698730469,-73.940811157226563,40.809253692626953,1,1.40,6,0.5,0.5,0,0,,,7,2,1,75,42,green,0.00,0.0,0.0,46,22,5.59,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1184,208,1,Manhattan,020800,1020800,E,MN03,Central Harlem North-Polo Grounds,3803
|
||||
3389228,1,2014-03-28 13:52:56,2014-03-28 14:29:01,N,1,-73.950569152832031,40.792312622070313,-73.868507385253906,40.688491821289063,2,16.10,46,0,0.5,0,5.33,,,51.83,2,,75,63,green,0.04,0.0,0.0,62,37,5.37,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1544,1182.02,3,Brooklyn,118202,3118202,E,BK83,Cypress Hills-City Line,4008
|
||||
3389229,2,2014-03-07 09:46:32,2014-03-07 09:55:01,N,1,-73.952301025390625,40.789798736572266,-73.935806274414062,40.794448852539063,1,1.67,8,0,0.5,2,0,,,10.5,1,1,75,74,green,0.00,3.9,0.0,37,26,7.83,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1553,178,1,Manhattan,017800,1017800,E,MN34,East Harlem North,3804
|
||||
3389230,2,2014-03-17 18:23:05,2014-03-17 18:28:38,N,1,-73.952346801757813,40.789844512939453,-73.946319580078125,40.783851623535156,5,0.95,5.5,1,0.5,0.65,0,,,7.65,1,1,75,263,green,0.00,0.0,0.0,35,23,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,32,156.01,1,Manhattan,015601,1015601,I,MN32,Yorkville,3805
|
||||
3389231,1,2014-03-19 19:09:36,2014-03-19 19:12:20,N,1,-73.952377319335938,40.789779663085938,-73.947494506835938,40.796474456787109,1,0.50,4,1,0.5,1,0,,,6.5,1,,75,75,green,0.92,0.0,0.0,46,32,7.16,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1401,174.02,1,Manhattan,017402,1017402,E,MN33,East Harlem South,3804
|
||||
3389232,2,2014-03-20 19:06:28,2014-03-20 19:21:35,N,1,-73.952583312988281,40.789516448974609,-73.985870361328125,40.776973724365234,2,3.04,13,1,0.5,2.8,0,,,17.3,1,1,75,143,green,0.00,0.0,0.0,54,40,8.05,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1742,155,1,Manhattan,015500,1015500,I,MN14,Lincoln Square,3806
|
||||
3389233,2,2014-03-29 09:38:12,2014-03-29 09:44:16,N,1,-73.952728271484375,40.789501190185547,-73.950935363769531,40.775600433349609,1,1.10,6.5,0,0.5,1.3,0,,,8.3,1,1,75,263,green,1.81,0.0,0.0,59,43,10.74,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,2048,138,1,Manhattan,013800,1013800,I,MN32,Yorkville,3805
|
||||
`,
|
||||
totalFields: 308 * 2,
|
||||
},
|
||||
}
|
||||
|
||||
for i, c := range cases {
|
||||
t.Run(c.file, func(t *testing.T) {
|
||||
|
||||
var err error
|
||||
var record sql.Record
|
||||
var result bytes.Buffer
|
||||
input := openTestFile(t, c.file)
|
||||
// Get above block size.
|
||||
input = append(input, input...)
|
||||
args := ReaderArgs{
|
||||
FileHeaderInfo: use,
|
||||
RecordDelimiter: c.recordDelimiter,
|
||||
FieldDelimiter: c.fieldDelimiter,
|
||||
QuoteCharacter: defaultQuoteCharacter,
|
||||
QuoteEscapeCharacter: defaultQuoteEscapeCharacter,
|
||||
CommentCharacter: defaultCommentCharacter,
|
||||
AllowQuotedRecordDelimiter: false,
|
||||
unmarshaled: true,
|
||||
}
|
||||
if !c.header {
|
||||
args.FileHeaderInfo = none
|
||||
}
|
||||
r, _ := NewReader(ioutil.NopCloser(bytes.NewReader(input)), &args)
|
||||
fields := 0
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
if fields < 10 {
|
||||
opts := sql.WriteCSVOpts{
|
||||
FieldDelimiter: ',',
|
||||
Quote: '"',
|
||||
QuoteEscape: '"',
|
||||
AlwaysQuote: false,
|
||||
}
|
||||
// Write with fixed delimiters, newlines.
|
||||
err := record.WriteCSV(&result, opts)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
}
|
||||
fields++
|
||||
}
|
||||
r.Close()
|
||||
if err != io.EOF {
|
||||
t.Fatalf("Case %d failed with %s", i, err)
|
||||
}
|
||||
if !reflect.DeepEqual(r.columnNames, c.wantColumns) {
|
||||
t.Errorf("Case %d failed: expected %#v, got result %#v", i, c.wantColumns, r.columnNames)
|
||||
}
|
||||
if result.String() != c.wantTenFields {
|
||||
t.Errorf("Case %d failed: expected %v, got result %v", i, c.wantTenFields, result.String())
|
||||
}
|
||||
if fields != c.totalFields {
|
||||
t.Errorf("Case %d failed: expected %v results %v", i, c.totalFields, fields)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type errReader struct {
|
||||
err error
|
||||
}
|
||||
|
||||
func (e errReader) Read(p []byte) (n int, err error) {
|
||||
return 0, e.err
|
||||
}
|
||||
|
||||
func TestReadFailures(t *testing.T) {
|
||||
customErr := errors.New("unable to read file :(")
|
||||
cases := []struct {
|
||||
file string
|
||||
recordDelimiter string
|
||||
fieldDelimiter string
|
||||
sendErr error
|
||||
header bool
|
||||
wantColumns []string
|
||||
wantFields string
|
||||
wantErr error
|
||||
}{
|
||||
{
|
||||
file: "truncated-records.csv",
|
||||
recordDelimiter: "^Y",
|
||||
fieldDelimiter: ",",
|
||||
header: true,
|
||||
wantColumns: []string{"trip_id", "vendor_id", "pickup_datetime", "dropoff_datetime", "store_and_fwd_flag", "rate_code_id", "pickup_longitude", "pickup_latitude", "dropoff_longitude", "dropoff_latitude", "passenger_count", "trip_distance", "fare_amount", "extra", "mta_tax", "tip_amount", "tolls_amount", "ehail_fee", "improvement_surcharge", "total_amount", "payment_type", "trip_type", "pickup", "dropoff", "cab_type", "precipitation", "snow_depth", "snowfall", "max_temp", "min_temp", "wind", "pickup_nyct2010_gid", "pickup_ctlabel", "pickup_borocode", "pickup_boroname", "pickup_ct2010", "pickup_boroct2010", "pickup_cdeligibil", "pickup_ntacode", "pickup_ntaname", "pickup_puma", "dropoff_nyct2010_gid", "dropoff_ctlabel", "dropoff_borocode", "dropoff_boroname", "dropoff_ct2010", "dropoff_boroct2010", "dropoff_cdeligibil", "dropoff_ntacode", "dropoff_ntaname", "dropoff_puma"},
|
||||
wantFields: `3389224,2,2014-03-26 00:26:15,2014-03-26 00:28:38,N,1,-73.950431823730469,40.792251586914063,-73.938949584960937,40.794425964355469,1,0.84,4.5,0.5,0.5,1,0,,,6.5,1,1,75,74,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1828,180,1,Manhattan,018000,1018000,E,MN34,East Harlem North,3804
|
||||
3389225,2,2014-03-31 09:42:15,2014-03-31 10:01:17,N,1,-73.950340270996094,40.792228698730469,-73.941970825195313,40.842235565185547,1,4.47,17.5,0,0.5,0,0,,,18,2,1,75,244,green,0.16,0.0,0.0,56,36,8.28,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,911,251,1,Manhattan,025100
|
||||
`,
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
file: "truncated-records.csv",
|
||||
recordDelimiter: "^Y",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: customErr,
|
||||
header: true,
|
||||
wantColumns: []string{"trip_id", "vendor_id", "pickup_datetime", "dropoff_datetime", "store_and_fwd_flag", "rate_code_id", "pickup_longitude", "pickup_latitude", "dropoff_longitude", "dropoff_latitude", "passenger_count", "trip_distance", "fare_amount", "extra", "mta_tax", "tip_amount", "tolls_amount", "ehail_fee", "improvement_surcharge", "total_amount", "payment_type", "trip_type", "pickup", "dropoff", "cab_type", "precipitation", "snow_depth", "snowfall", "max_temp", "min_temp", "wind", "pickup_nyct2010_gid", "pickup_ctlabel", "pickup_borocode", "pickup_boroname", "pickup_ct2010", "pickup_boroct2010", "pickup_cdeligibil", "pickup_ntacode", "pickup_ntaname", "pickup_puma", "dropoff_nyct2010_gid", "dropoff_ctlabel", "dropoff_borocode", "dropoff_boroname", "dropoff_ct2010", "dropoff_boroct2010", "dropoff_cdeligibil", "dropoff_ntacode", "dropoff_ntaname", "dropoff_puma"},
|
||||
wantFields: `3389224,2,2014-03-26 00:26:15,2014-03-26 00:28:38,N,1,-73.950431823730469,40.792251586914063,-73.938949584960937,40.794425964355469,1,0.84,4.5,0.5,0.5,1,0,,,6.5,1,1,75,74,green,0.00,0.0,0.0,36,24,11.86,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,1828,180,1,Manhattan,018000,1018000,E,MN34,East Harlem North,3804
|
||||
3389225,2,2014-03-31 09:42:15,2014-03-31 10:01:17,N,1,-73.950340270996094,40.792228698730469,-73.941970825195313,40.842235565185547,1,4.47,17.5,0,0.5,0,0,,,18,2,1,75,244,green,0.16,0.0,0.0,56,36,8.28,1267,168,1,Manhattan,016800,1016800,E,MN33,East Harlem South,3804,911,251,1,Manhattan,025100
|
||||
`,
|
||||
wantErr: customErr,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-badbarequote.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `"a ""word""",b` + "\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-baddoubleq.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `"a""""b",c` + "\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-badextraq.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `a word,"b"""` + "\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-badstartline.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `a,"b` + "\n" + `c""d,e` + "\n\"\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-badstartline2.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `a,b` + "\n" + `"d` + "\n\ne\"\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-badtrailingq.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `a word,"b"""` + "\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-crlfquoted.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `"foo""bar"` + "\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true:
|
||||
file: "invalid-csv.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `"a""""b",c` + "\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// This works since LazyQuotes is true, but output is very weird.
|
||||
file: "invalid-oddquote.csv",
|
||||
recordDelimiter: "\n",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: true,
|
||||
wantColumns: []string{"header1", "header2", "header3"},
|
||||
wantFields: "ok1,ok2,ok3\n" + `""""""",b,c` + "\n\"\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
{
|
||||
// Test when file ends with a half separator
|
||||
file: "endswithhalfsep.csv",
|
||||
recordDelimiter: "%!",
|
||||
fieldDelimiter: ",",
|
||||
sendErr: nil,
|
||||
header: false,
|
||||
wantColumns: []string{"_1", "_2", "_3"},
|
||||
wantFields: "a,b,c\na2,b2,c2%\n",
|
||||
wantErr: io.EOF,
|
||||
},
|
||||
}
|
||||
|
||||
for i, c := range cases {
|
||||
t.Run(c.file, func(t *testing.T) {
|
||||
|
||||
var err error
|
||||
var record sql.Record
|
||||
var result bytes.Buffer
|
||||
input := openTestFile(t, c.file)
|
||||
args := ReaderArgs{
|
||||
FileHeaderInfo: use,
|
||||
RecordDelimiter: c.recordDelimiter,
|
||||
FieldDelimiter: c.fieldDelimiter,
|
||||
QuoteCharacter: defaultQuoteCharacter,
|
||||
QuoteEscapeCharacter: defaultQuoteEscapeCharacter,
|
||||
CommentCharacter: defaultCommentCharacter,
|
||||
AllowQuotedRecordDelimiter: false,
|
||||
unmarshaled: true,
|
||||
}
|
||||
if !c.header {
|
||||
args.FileHeaderInfo = none
|
||||
}
|
||||
inr := io.Reader(bytes.NewReader(input))
|
||||
if c.sendErr != nil {
|
||||
inr = io.MultiReader(inr, errReader{c.sendErr})
|
||||
}
|
||||
r, _ := NewReader(ioutil.NopCloser(inr), &args)
|
||||
fields := 0
|
||||
for {
|
||||
record, err = r.Read(record)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
|
||||
opts := sql.WriteCSVOpts{
|
||||
FieldDelimiter: ',',
|
||||
Quote: '"',
|
||||
QuoteEscape: '"',
|
||||
AlwaysQuote: false,
|
||||
}
|
||||
// Write with fixed delimiters, newlines.
|
||||
err := record.WriteCSV(&result, opts)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
fields++
|
||||
}
|
||||
r.Close()
|
||||
if err != c.wantErr {
|
||||
t.Fatalf("Case %d failed with %s", i, err)
|
||||
}
|
||||
if !reflect.DeepEqual(r.columnNames, c.wantColumns) {
|
||||
t.Errorf("Case %d failed: expected \n%#v, got result \n%#v", i, c.wantColumns, r.columnNames)
|
||||
}
|
||||
if result.String() != c.wantFields {
|
||||
t.Errorf("Case %d failed: expected \n%v\nGot result \n%v", i, c.wantFields, result.String())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkReaderBasic(b *testing.B) {
|
||||
args := ReaderArgs{
|
||||
FileHeaderInfo: use,
|
||||
RecordDelimiter: "\n",
|
||||
FieldDelimiter: ",",
|
||||
QuoteCharacter: defaultQuoteCharacter,
|
||||
QuoteEscapeCharacter: defaultQuoteEscapeCharacter,
|
||||
CommentCharacter: defaultCommentCharacter,
|
||||
AllowQuotedRecordDelimiter: false,
|
||||
unmarshaled: true,
|
||||
}
|
||||
f := openTestFile(b, "nyc-taxi-data-100k.csv")
|
||||
r, err := NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &args)
|
||||
if err != nil {
|
||||
b.Fatalf("Reading init failed with %s", err)
|
||||
}
|
||||
defer r.Close()
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
b.SetBytes(int64(len(f)))
|
||||
var record sql.Record
|
||||
for i := 0; i < b.N; i++ {
|
||||
r, err = NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &args)
|
||||
if err != nil {
|
||||
b.Fatalf("Reading init failed with %s", err)
|
||||
}
|
||||
for err == nil {
|
||||
record, err = r.Read(record)
|
||||
if err != nil && err != io.EOF {
|
||||
b.Fatalf("Reading failed with %s", err)
|
||||
}
|
||||
}
|
||||
r.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkReaderHuge(b *testing.B) {
|
||||
args := ReaderArgs{
|
||||
FileHeaderInfo: use,
|
||||
RecordDelimiter: "\n",
|
||||
FieldDelimiter: ",",
|
||||
QuoteCharacter: defaultQuoteCharacter,
|
||||
QuoteEscapeCharacter: defaultQuoteEscapeCharacter,
|
||||
CommentCharacter: defaultCommentCharacter,
|
||||
AllowQuotedRecordDelimiter: false,
|
||||
unmarshaled: true,
|
||||
}
|
||||
for n := 0; n < 11; n++ {
|
||||
f := openTestFile(b, "nyc-taxi-data-100k.csv")
|
||||
want := 309
|
||||
for i := 0; i < n; i++ {
|
||||
f = append(f, f...)
|
||||
want *= 2
|
||||
}
|
||||
b.Run(fmt.Sprint(len(f)/(1<<10), "K"), func(b *testing.B) {
|
||||
b.ReportAllocs()
|
||||
b.SetBytes(int64(len(f)))
|
||||
b.ResetTimer()
|
||||
var record sql.Record
|
||||
for i := 0; i < b.N; i++ {
|
||||
r, err := NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &args)
|
||||
if err != nil {
|
||||
b.Fatalf("Reading init failed with %s", err)
|
||||
}
|
||||
|
||||
got := 0
|
||||
for err == nil {
|
||||
record, err = r.Read(record)
|
||||
if err != nil && err != io.EOF {
|
||||
b.Fatalf("Reading failed with %s", err)
|
||||
}
|
||||
got++
|
||||
}
|
||||
r.Close()
|
||||
if got != want {
|
||||
b.Errorf("want %d records, got %d", want, got)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkReaderReplace(b *testing.B) {
|
||||
args := ReaderArgs{
|
||||
FileHeaderInfo: use,
|
||||
RecordDelimiter: "^",
|
||||
FieldDelimiter: ",",
|
||||
QuoteCharacter: defaultQuoteCharacter,
|
||||
QuoteEscapeCharacter: defaultQuoteEscapeCharacter,
|
||||
CommentCharacter: defaultCommentCharacter,
|
||||
AllowQuotedRecordDelimiter: false,
|
||||
unmarshaled: true,
|
||||
}
|
||||
f := openTestFile(b, "nyc-taxi-data-100k-single-delim.csv")
|
||||
r, err := NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &args)
|
||||
if err != nil {
|
||||
b.Fatalf("Reading init failed with %s", err)
|
||||
}
|
||||
defer r.Close()
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
b.SetBytes(int64(len(f)))
|
||||
var record sql.Record
|
||||
for i := 0; i < b.N; i++ {
|
||||
r, err = NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &args)
|
||||
if err != nil {
|
||||
b.Fatalf("Reading init failed with %s", err)
|
||||
}
|
||||
|
||||
for err == nil {
|
||||
record, err = r.Read(record)
|
||||
if err != nil && err != io.EOF {
|
||||
b.Fatalf("Reading failed with %s", err)
|
||||
}
|
||||
}
|
||||
r.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkReaderReplaceTwo(b *testing.B) {
|
||||
args := ReaderArgs{
|
||||
FileHeaderInfo: use,
|
||||
RecordDelimiter: "^Y",
|
||||
FieldDelimiter: ",",
|
||||
QuoteCharacter: defaultQuoteCharacter,
|
||||
QuoteEscapeCharacter: defaultQuoteEscapeCharacter,
|
||||
CommentCharacter: defaultCommentCharacter,
|
||||
AllowQuotedRecordDelimiter: false,
|
||||
unmarshaled: true,
|
||||
}
|
||||
f := openTestFile(b, "nyc-taxi-data-100k-multi-delim.csv")
|
||||
r, err := NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &args)
|
||||
if err != nil {
|
||||
b.Fatalf("Reading init failed with %s", err)
|
||||
}
|
||||
defer r.Close()
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
b.SetBytes(int64(len(f)))
|
||||
var record sql.Record
|
||||
for i := 0; i < b.N; i++ {
|
||||
r, err = NewReader(ioutil.NopCloser(bytes.NewBuffer(f)), &args)
|
||||
if err != nil {
|
||||
b.Fatalf("Reading init failed with %s", err)
|
||||
}
|
||||
|
||||
for err == nil {
|
||||
record, err = r.Read(record)
|
||||
if err != nil && err != io.EOF {
|
||||
b.Fatalf("Reading failed with %s", err)
|
||||
}
|
||||
}
|
||||
r.Close()
|
||||
}
|
||||
}
|
||||
131
internal/s3select/csv/record.go
Normal file
131
internal/s3select/csv/record.go
Normal file
@@ -0,0 +1,131 @@
|
||||
// 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 csv
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
"github.com/bcicen/jstream"
|
||||
csv "github.com/minio/csvparser"
|
||||
"github.com/minio/minio/internal/s3select/sql"
|
||||
)
|
||||
|
||||
// Record - is a CSV record.
|
||||
type Record struct {
|
||||
columnNames []string
|
||||
csvRecord []string
|
||||
nameIndexMap map[string]int64
|
||||
}
|
||||
|
||||
// Get - gets the value for a column name. CSV fields do not have any
|
||||
// defined type (other than the default string). So this function
|
||||
// always returns fields using sql.FromBytes so that the type
|
||||
// specified/implied by the query can be used, or can be automatically
|
||||
// converted based on the query.
|
||||
func (r *Record) Get(name string) (*sql.Value, error) {
|
||||
index, found := r.nameIndexMap[name]
|
||||
if !found {
|
||||
return nil, fmt.Errorf("column %v not found", name)
|
||||
}
|
||||
|
||||
if index >= int64(len(r.csvRecord)) {
|
||||
// No value found for column 'name', hence return null
|
||||
// value
|
||||
return sql.FromNull(), nil
|
||||
}
|
||||
|
||||
return sql.FromBytes([]byte(r.csvRecord[index])), nil
|
||||
}
|
||||
|
||||
// Set - sets the value for a column name.
|
||||
func (r *Record) Set(name string, value *sql.Value) (sql.Record, error) {
|
||||
r.columnNames = append(r.columnNames, name)
|
||||
r.csvRecord = append(r.csvRecord, value.CSVString())
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// Reset data in record.
|
||||
func (r *Record) Reset() {
|
||||
if len(r.columnNames) > 0 {
|
||||
r.columnNames = r.columnNames[:0]
|
||||
}
|
||||
if len(r.csvRecord) > 0 {
|
||||
r.csvRecord = r.csvRecord[:0]
|
||||
}
|
||||
for k := range r.nameIndexMap {
|
||||
delete(r.nameIndexMap, k)
|
||||
}
|
||||
}
|
||||
|
||||
// Clone the record.
|
||||
func (r *Record) Clone(dst sql.Record) sql.Record {
|
||||
other, ok := dst.(*Record)
|
||||
if !ok {
|
||||
other = &Record{}
|
||||
}
|
||||
if len(other.columnNames) > 0 {
|
||||
other.columnNames = other.columnNames[:0]
|
||||
}
|
||||
if len(other.csvRecord) > 0 {
|
||||
other.csvRecord = other.csvRecord[:0]
|
||||
}
|
||||
other.columnNames = append(other.columnNames, r.columnNames...)
|
||||
other.csvRecord = append(other.csvRecord, r.csvRecord...)
|
||||
return other
|
||||
}
|
||||
|
||||
// WriteCSV - encodes to CSV data.
|
||||
func (r *Record) WriteCSV(writer io.Writer, opts sql.WriteCSVOpts) error {
|
||||
w := csv.NewWriter(writer)
|
||||
w.Comma = opts.FieldDelimiter
|
||||
w.AlwaysQuote = opts.AlwaysQuote
|
||||
w.Quote = opts.Quote
|
||||
w.QuoteEscape = opts.QuoteEscape
|
||||
if err := w.Write(r.csvRecord); err != nil {
|
||||
return err
|
||||
}
|
||||
w.Flush()
|
||||
return w.Error()
|
||||
}
|
||||
|
||||
// WriteJSON - encodes to JSON data.
|
||||
func (r *Record) WriteJSON(writer io.Writer) error {
|
||||
var kvs jstream.KVS = make([]jstream.KV, len(r.columnNames))
|
||||
for i := 0; i < len(r.columnNames); i++ {
|
||||
kvs[i] = jstream.KV{Key: r.columnNames[i], Value: r.csvRecord[i]}
|
||||
}
|
||||
return json.NewEncoder(writer).Encode(kvs)
|
||||
}
|
||||
|
||||
// Raw - returns the underlying data with format info.
|
||||
func (r *Record) Raw() (sql.SelectObjectFormat, interface{}) {
|
||||
return sql.SelectFmtCSV, r
|
||||
}
|
||||
|
||||
// Replace - is not supported for CSV
|
||||
func (r *Record) Replace(_ interface{}) error {
|
||||
return errors.New("Replace is not supported for CSV")
|
||||
}
|
||||
|
||||
// NewRecord - creates new CSV record.
|
||||
func NewRecord() *Record {
|
||||
return &Record{}
|
||||
}
|
||||
94
internal/s3select/csv/recordtransform.go
Normal file
94
internal/s3select/csv/recordtransform.go
Normal file
@@ -0,0 +1,94 @@
|
||||
// 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 csv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"io"
|
||||
)
|
||||
|
||||
// recordTransform will convert records to always have newline records.
|
||||
type recordTransform struct {
|
||||
reader io.Reader
|
||||
// recordDelimiter can be up to 2 characters.
|
||||
recordDelimiter []byte
|
||||
oneByte []byte
|
||||
useOneByte bool
|
||||
}
|
||||
|
||||
func (rr *recordTransform) Read(p []byte) (n int, err error) {
|
||||
if rr.useOneByte {
|
||||
p[0] = rr.oneByte[0]
|
||||
rr.useOneByte = false
|
||||
n, err = rr.reader.Read(p[1:])
|
||||
n++
|
||||
} else {
|
||||
n, err = rr.reader.Read(p)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
// Do nothing if record-delimiter is already newline.
|
||||
if string(rr.recordDelimiter) == "\n" {
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// Change record delimiters to newline.
|
||||
if len(rr.recordDelimiter) == 1 {
|
||||
for idx := 0; idx < len(p); {
|
||||
i := bytes.Index(p[idx:], rr.recordDelimiter)
|
||||
if i < 0 {
|
||||
break
|
||||
}
|
||||
idx += i
|
||||
p[idx] = '\n'
|
||||
}
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// 2 characters...
|
||||
for idx := 0; idx < len(p); {
|
||||
i := bytes.Index(p[idx:], rr.recordDelimiter)
|
||||
if i < 0 {
|
||||
break
|
||||
}
|
||||
idx += i
|
||||
|
||||
p[idx] = '\n'
|
||||
p = append(p[:idx+1], p[idx+2:]...)
|
||||
n--
|
||||
}
|
||||
|
||||
if p[n-1] != rr.recordDelimiter[0] {
|
||||
return n, nil
|
||||
}
|
||||
|
||||
if _, err = rr.reader.Read(rr.oneByte); err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
if rr.oneByte[0] == rr.recordDelimiter[1] {
|
||||
p[n-1] = '\n'
|
||||
return n, nil
|
||||
}
|
||||
|
||||
rr.useOneByte = true
|
||||
return n, nil
|
||||
}
|
||||
BIN
internal/s3select/csv/testdata/testdata.zip
vendored
Normal file
BIN
internal/s3select/csv/testdata/testdata.zip
vendored
Normal file
Binary file not shown.
Reference in New Issue
Block a user