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:
Harshavardhana
2021-06-01 14:59:40 -07:00
committed by GitHub
parent bf87c4b1e4
commit 1f262daf6f
540 changed files with 757 additions and 778 deletions

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

View 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"),
}
}

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

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

View 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{}
}

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

Binary file not shown.